summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorØyvind Grønnesby <oyving@yahooinc.com>2023-03-30 05:25:54 -0700
committerØyvind Grønnesby <oyving@yahooinc.com>2023-03-30 05:25:54 -0700
commitf86324ef8662bda7bf11bacbcc2c84e07e1b813e (patch)
treeb82c492a292376b65cd0251badda7a5afba74c35
parent1e8cf546bfa39d23c672350cf44b75b2ea4f23ee (diff)
parent3fb3c3898e8234d2e2fd8a086b4b341b27977cbf (diff)
Merge remote-tracking branch 'origin/master' into ogronnesby/gpu-billing
-rw-r--r--client/go/internal/cli/cmd/feed.go8
-rw-r--r--client/go/internal/cli/cmd/prod.go3
-rw-r--r--client/go/internal/vespa/document/circuit_breaker.go77
-rw-r--r--client/go/internal/vespa/document/circuit_breaker_test.go49
-rw-r--r--client/go/internal/vespa/document/dispatcher.go158
-rw-r--r--client/go/internal/vespa/document/dispatcher_test.go53
-rw-r--r--client/go/internal/vespa/document/feeder.go25
-rw-r--r--client/go/internal/vespa/document/feeder_test.go2
-rw-r--r--client/go/internal/vespa/document/http.go61
-rw-r--r--client/go/internal/vespa/document/http_test.go10
-rw-r--r--client/go/internal/vespa/document/throttler.go117
-rw-r--r--client/go/internal/vespa/document/throttler_test.go21
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/ContentCluster.java12
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeChecker.java18
-rw-r--r--clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequest.java4
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeCheckerTest.java137
-rw-r--r--clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequestTest.java10
-rw-r--r--config-model-api/src/main/java/com/yahoo/config/model/api/ModelContext.java3
-rw-r--r--config-model/src/main/java/com/yahoo/config/model/deploy/TestProperties.java6
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java50
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/container/search/QueryProfiles.java12
-rw-r--r--config-model/src/main/java/com/yahoo/vespa/model/search/IndexedSearchCluster.java11
-rw-r--r--config-model/src/main/javacc/SchemaParser.jj304
-rw-r--r--config-model/src/test/derived/globalphase_onnx_inside/onnx-models.cfg2
-rw-r--r--config-model/src/test/derived/globalphase_onnx_inside/test.sd1
-rw-r--r--config-model/src/test/java/com/yahoo/vespa/model/content/ContentClusterTest.java24
-rw-r--r--config-provisioning/src/main/java/com/yahoo/config/provision/zone/ZoneApi.java3
-rw-r--r--config/src/vespa/config/frt/frtconnection.cpp4
-rw-r--r--configdefinitions/src/vespa/dispatch.def26
-rw-r--r--configdefinitions/src/vespa/stor-distribution.def3
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ModelContextImpl.java3
-rw-r--r--container-core/src/main/java/com/yahoo/processing/Request.java4
-rw-r--r--container-core/src/main/java/com/yahoo/processing/Response.java2
-rw-r--r--container-core/src/main/java/com/yahoo/processing/handler/AbstractProcessingHandler.java9
-rw-r--r--container-core/src/main/java/com/yahoo/processing/request/CompoundName.java69
-rw-r--r--container-core/src/test/java/com/yahoo/processing/request/CompoundNameTestCase.java112
-rw-r--r--container-core/src/test/java/com/yahoo/processing/request/test/RequestTestCase.java85
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/IndexFacts.java98
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java62
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/cluster/SchemaResolver.java58
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/query/parser/CustomParser.java3
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/query/parser/Tokenizer.java14
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/querytransform/NonPhrasingSearcher.java2
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/querytransform/PhrasingSearcher.java2
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/querytransform/RecallSearcher.java2
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/querytransform/StemmingSearcher.java62
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/searcher/FieldCollapsingSearcher.java8
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/searcher/JSONDebugSearcher.java2
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/searcher/MultipleResultsSearcher.java22
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/searcher/PosSearcher.java23
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/searcher/ValidateSortingSearcher.java4
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/semantics/SemanticSearcher.java6
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/Query.java70
-rw-r--r--container-search/src/main/java/com/yahoo/search/dispatch/Dispatcher.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/dispatch/rpc/CompressService.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcInvokerFactory.java12
-rw-r--r--container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcProtobufFillInvoker.java12
-rw-r--r--container-search/src/main/java/com/yahoo/search/federation/FederationSearcher.java31
-rw-r--r--container-search/src/main/java/com/yahoo/search/grouping/GroupingQueryParser.java15
-rw-r--r--container-search/src/main/java/com/yahoo/search/grouping/GroupingValidator.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/grouping/UniqueGroupingSearcher.java39
-rw-r--r--container-search/src/main/java/com/yahoo/search/grouping/result/FlatteningSearcher.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/grouping/vespa/GroupingExecutor.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/handler/SearchHandler.java4
-rw-r--r--container-search/src/main/java/com/yahoo/search/intent/model/IntentModel.java13
-rw-r--r--container-search/src/main/java/com/yahoo/search/pagetemplates/PageTemplateSearcher.java8
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/Model.java4
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/Ranking.java4
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/profile/compiled/CompiledQueryProfile.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/profile/types/FieldDescription.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/profile/types/QueryProfileType.java8
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/properties/DefaultProperties.java6
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/properties/SubProperties.java4
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/ranking/SoftTimeout.java3
-rw-r--r--container-search/src/main/java/com/yahoo/search/query/rewrite/RewriterConstants.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/querytransform/BooleanSearcher.java6
-rw-r--r--container-search/src/main/java/com/yahoo/search/querytransform/SortingDegrader.java4
-rw-r--r--container-search/src/main/java/com/yahoo/search/querytransform/WandSearcher.java12
-rw-r--r--container-search/src/main/java/com/yahoo/search/querytransform/WeakAndReplacementSearcher.java4
-rw-r--r--container-search/src/main/java/com/yahoo/search/ranking/GlobalPhaseRanker.java31
-rw-r--r--container-search/src/main/java/com/yahoo/search/rendering/JsonRenderer.java8
-rw-r--r--container-search/src/main/java/com/yahoo/search/searchers/CacheControlSearcher.java6
-rw-r--r--container-search/src/main/java/com/yahoo/search/searchers/ConnectionControlSearcher.java2
-rwxr-xr-xcontainer-search/src/main/java/com/yahoo/search/searchers/RateLimitingSearcher.java10
-rw-r--r--container-search/src/main/java/com/yahoo/search/yql/FieldFiller.java2
-rw-r--r--container-search/src/main/java/com/yahoo/search/yql/FieldFilter.java4
-rw-r--r--container-search/src/main/java/com/yahoo/search/yql/MinimalQueryInserter.java6
-rw-r--r--container-search/src/main/java/com/yahoo/vespa/streamingvisitors/MetricsSearcher.java7
-rw-r--r--container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java6
-rw-r--r--container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsVisitor.java14
-rw-r--r--container-search/src/test/java/com/yahoo/prelude/query/parser/test/TokenizerTestCase.java15
-rw-r--r--container-search/src/test/java/com/yahoo/prelude/test/IndexFactsTestCase.java8
-rw-r--r--container-search/src/test/java/com/yahoo/search/query/profile/config/test/XmlReadingTestCase.java32
-rw-r--r--container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetInComplexStructureMicroBenchmark.java4
-rw-r--r--container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetMicroBenchmark.java4
-rw-r--r--container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileTestCase.java33
-rw-r--r--container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileVariantsTestCase.java13
-rw-r--r--container-search/src/test/java/com/yahoo/search/query/profile/types/test/NameTestCase.java6
-rw-r--r--container-search/src/test/java/com/yahoo/search/query/profile/types/test/QueryProfileTypeTestCase.java2
-rw-r--r--container-search/src/test/java/com/yahoo/search/query/properties/SubPropertiesTestCase.java (renamed from container-search/src/test/java/com/yahoo/search/query/properties/test/SubPropertiesTestCase.java)5
-rw-r--r--container-search/src/test/java/com/yahoo/search/test/QueryTestCase.java14
-rw-r--r--controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/zone/ZoneRegistry.java4
-rw-r--r--controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java10
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneApiMock.java15
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java15
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ArchiveAccessMaintainerTest.java6
-rw-r--r--controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-cloud.json3
-rw-r--r--flags/src/main/java/com/yahoo/vespa/flags/Flags.java6
-rw-r--r--parent/pom.xml2
-rw-r--r--screwdriver.yaml2
-rwxr-xr-xscrewdriver/release-container-image.sh3
-rw-r--r--searchcore/CMakeLists.txt1
-rw-r--r--searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp10
-rw-r--r--searchcore/src/tests/proton/attribute/attributes_state_explorer/attributes_state_explorer_test.cpp4
-rw-r--r--searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/.gitignore1
-rw-r--r--searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/CMakeLists.txt9
-rw-r--r--searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/exclusive_attribute_read_accessor_test.cpp55
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/CMakeLists.txt2
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/attribute_executor.cpp34
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/attribute_executor.h32
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/attribute_manager_explorer.cpp15
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.cpp40
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.h10
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/attributemanager.cpp10
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/attributemanager.h2
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.cpp60
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.h53
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.cpp6
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.h1
-rw-r--r--searchcore/src/vespa/searchcore/proton/attribute/i_attribute_manager.h3
-rw-r--r--searchcore/src/vespa/searchcore/proton/documentmetastore/document_meta_store_explorer.cpp9
-rw-r--r--searchcore/src/vespa/searchcore/proton/test/mock_attribute_manager.h3
-rw-r--r--searchlib/src/vespa/searchlib/util/state_explorer_utils.cpp26
-rw-r--r--searchlib/src/vespa/searchlib/util/state_explorer_utils.h2
-rw-r--r--storage/src/vespa/storage/frameworkimpl/thread/appkiller.h2
-rw-r--r--storage/src/vespa/storage/storageserver/servicelayernode.h2
-rw-r--r--storage/src/vespa/storage/storageserver/storagenode.cpp2
-rw-r--r--storage/src/vespa/storage/storageserver/storagenode.h8
-rw-r--r--storage/src/vespa/storageframework/generic/thread/thread.h10
-rw-r--r--vdslib/src/main/java/com/yahoo/vdslib/distribution/Distribution.java23
-rw-r--r--vdslib/src/test/java/com/yahoo/vdslib/distribution/DistributionTestCase.java30
-rw-r--r--vdslib/src/tests/distribution/testdata/hierarchical-grouping-distributor-notakeover.java.results585
-rw-r--r--vespa-dependencies-enforcer/allowed-maven-dependencies.txt2
-rw-r--r--vespajlib/src/main/java/com/yahoo/slime/BinaryDecoder.java53
-rw-r--r--vespajlib/src/main/java/com/yahoo/slime/BinaryEncoder.java14
-rw-r--r--vespajlib/src/main/java/com/yahoo/slime/BinaryView.java307
-rw-r--r--vespajlib/src/main/java/com/yahoo/slime/BufferedInput.java30
-rw-r--r--vespajlib/src/main/java/com/yahoo/slime/Slime.java2
-rw-r--r--vespajlib/src/main/java/com/yahoo/slime/Value.java4
-rw-r--r--vespajlib/src/test/java/com/yahoo/slime/BinaryFormatTestCase.java141
-rw-r--r--vespajlib/src/test/java/com/yahoo/slime/BinaryViewTest.java418
-rw-r--r--vespalib/src/vespa/vespalib/util/optimized.h8
-rw-r--r--zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Configurator.java156
-rw-r--r--zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Reconfigurer.java50
-rw-r--r--zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ConfiguratorTest.java49
-rw-r--r--zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ReconfigurerTest.java12
157 files changed, 2700 insertions, 2037 deletions
diff --git a/client/go/internal/cli/cmd/feed.go b/client/go/internal/cli/cmd/feed.go
index f273c5aa826..c8e032929b8 100644
--- a/client/go/internal/cli/cmd/feed.go
+++ b/client/go/internal/cli/cmd/feed.go
@@ -13,6 +13,7 @@ import (
)
func addFeedFlags(cmd *cobra.Command, concurrency *int) {
+ // TOOD(mpolden): Remove this flag
cmd.PersistentFlags().IntVarP(concurrency, "concurrency", "T", 64, "Number of goroutines to use for dispatching")
}
@@ -58,7 +59,10 @@ func feed(r io.Reader, cli *CLI, concurrency int) error {
client := document.NewClient(document.ClientOptions{
BaseURL: service.BaseURL,
}, service)
- dispatcher := document.NewDispatcher(client, concurrency)
+ throttler := document.NewThrottler()
+ // TODO(mpolden): Make doom duration configurable
+ circuitBreaker := document.NewCircuitBreaker(10*time.Second, 0)
+ dispatcher := document.NewDispatcher(client, throttler, circuitBreaker)
dec := document.NewDecoder(r)
start := cli.now()
@@ -78,7 +82,7 @@ func feed(r io.Reader, cli *CLI, concurrency int) error {
return err
}
elapsed := cli.now().Sub(start)
- return writeSummaryJSON(cli.Stdout, client.Stats(), elapsed)
+ return writeSummaryJSON(cli.Stdout, dispatcher.Stats(), elapsed)
}
type number float32
diff --git a/client/go/internal/cli/cmd/prod.go b/client/go/internal/cli/cmd/prod.go
index 57b7abe5b6e..0e00597221a 100644
--- a/client/go/internal/cli/cmd/prod.go
+++ b/client/go/internal/cli/cmd/prod.go
@@ -144,9 +144,6 @@ $ vespa prod submit`,
if err := verifyTests(cli, pkg); err != nil {
return err
}
- if !cli.isCI() {
- cli.printWarning("We recommend doing this only from a CD job", "See https://cloud.vespa.ai/en/getting-to-production")
- }
opts, err := cli.createDeploymentOptions(pkg, target)
if err != nil {
return err
diff --git a/client/go/internal/vespa/document/circuit_breaker.go b/client/go/internal/vespa/document/circuit_breaker.go
new file mode 100644
index 00000000000..17fc595d58f
--- /dev/null
+++ b/client/go/internal/vespa/document/circuit_breaker.go
@@ -0,0 +1,77 @@
+package document
+
+import (
+ "math"
+ "sync/atomic"
+ "time"
+)
+
+type CircuitState int
+
+const (
+ // CircuitClosed represents a closed circuit. Documents are processed successfully
+ CircuitClosed CircuitState = iota
+ // CircuitHalfOpen represents a half-open circuit. Some errors have happend, but processing may still recover
+ CircuitHalfOpen
+ // CircuitOpen represents a open circuit. Something is broken. We should no longer process documents
+ CircuitOpen
+)
+
+type CircuitBreaker interface {
+ Success()
+ Error(error)
+ State() CircuitState
+}
+
+type timeCircuitBreaker struct {
+ graceDuration time.Duration
+ doomDuration time.Duration
+
+ failingSinceMillis int64
+ lastError atomic.Value
+ halfOpen atomic.Value
+ open atomic.Value
+
+ now func() time.Time
+}
+
+func (b *timeCircuitBreaker) Success() {
+ atomic.StoreInt64(&b.failingSinceMillis, math.MaxInt64)
+ if !b.open.Load().(bool) {
+ b.halfOpen.CompareAndSwap(true, false)
+ }
+}
+
+func (b *timeCircuitBreaker) Error(err error) {
+ if atomic.CompareAndSwapInt64(&b.failingSinceMillis, math.MaxInt64, b.now().UnixMilli()) {
+ b.lastError.Store(err)
+ }
+}
+
+func (b *timeCircuitBreaker) State() CircuitState {
+ failingDuration := b.now().Sub(time.UnixMilli(atomic.LoadInt64(&b.failingSinceMillis)))
+ if failingDuration > b.graceDuration {
+ b.halfOpen.CompareAndSwap(false, true)
+ }
+ if b.doomDuration > 0 && failingDuration > b.doomDuration {
+ b.open.CompareAndSwap(false, true)
+ }
+ if b.open.Load().(bool) {
+ return CircuitOpen
+ } else if b.halfOpen.Load().(bool) {
+ return CircuitHalfOpen
+ }
+ return CircuitClosed
+}
+
+func NewCircuitBreaker(graceDuration, doomDuration time.Duration) *timeCircuitBreaker {
+ b := &timeCircuitBreaker{
+ graceDuration: graceDuration,
+ doomDuration: doomDuration,
+ now: time.Now,
+ failingSinceMillis: math.MaxInt64,
+ }
+ b.open.Store(false)
+ b.halfOpen.Store(false)
+ return b
+}
diff --git a/client/go/internal/vespa/document/circuit_breaker_test.go b/client/go/internal/vespa/document/circuit_breaker_test.go
new file mode 100644
index 00000000000..7a4fffaae27
--- /dev/null
+++ b/client/go/internal/vespa/document/circuit_breaker_test.go
@@ -0,0 +1,49 @@
+package document
+
+import (
+ "errors"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/assert"
+)
+
+func TestCircuitBreaker(t *testing.T) {
+ clock := &manualClock{}
+ breaker := NewCircuitBreaker(time.Second, time.Minute)
+ breaker.now = clock.now
+ err := errors.New("error")
+
+ assert.Equal(t, CircuitClosed, breaker.State(), "Initial state is closed")
+
+ clock.advance(100 * time.Second)
+ assert.Equal(t, CircuitClosed, breaker.State(), "State is closed after some time without activity")
+
+ breaker.Success()
+ assert.Equal(t, CircuitClosed, breaker.State(), "State is closed after a success")
+
+ clock.advance(100 * time.Second)
+ assert.Equal(t, CircuitClosed, breaker.State(), "State is closed some time after a success")
+
+ breaker.Error(err)
+ assert.Equal(t, CircuitClosed, breaker.State(), "State is closed right after a failure")
+
+ clock.advance(time.Second)
+ assert.Equal(t, CircuitClosed, breaker.State(), "State is closed until grace duration has passed")
+
+ clock.advance(time.Millisecond)
+ assert.Equal(t, CircuitHalfOpen, breaker.State(), "State is half-open when grace duration has passed")
+
+ breaker.Success()
+ assert.Equal(t, CircuitClosed, breaker.State(), "State is closed after a new success")
+
+ breaker.Error(err)
+ clock.advance(time.Minute)
+ assert.Equal(t, CircuitHalfOpen, breaker.State(), "State is half-open until doom duration has passed")
+
+ clock.advance(time.Millisecond)
+ assert.Equal(t, CircuitOpen, breaker.State(), "State is open when doom duration has passed")
+
+ breaker.Success()
+ assert.Equal(t, CircuitOpen, breaker.State(), "State remains open in spite of new successes")
+}
diff --git a/client/go/internal/vespa/document/dispatcher.go b/client/go/internal/vespa/document/dispatcher.go
index feb562a241a..7011ae7a9b6 100644
--- a/client/go/internal/vespa/document/dispatcher.go
+++ b/client/go/internal/vespa/document/dispatcher.go
@@ -3,25 +3,33 @@ package document
import (
"fmt"
"sync"
+ "sync/atomic"
+ "time"
)
const maxAttempts = 10
// Dispatcher dispatches documents from a queue to a Feeder.
type Dispatcher struct {
- workers int
- feeder Feeder
- ready chan Id
- inflight map[string]*documentGroup
+ feeder Feeder
+ throttler Throttler
+ circuitBreaker CircuitBreaker
+ stats Stats
+
+ closed bool
+ ready chan Id
+ results chan Result
+ inflight map[string]*documentGroup
+ inflightCount int64
+
mu sync.RWMutex
wg sync.WaitGroup
- closed bool
+ resultWg sync.WaitGroup
}
// documentGroup holds document operations which share their ID, and must be dispatched in order.
type documentGroup struct {
id Id
- failed bool
operations []documentOp
mu sync.Mutex
}
@@ -37,68 +45,97 @@ func (g *documentGroup) append(op documentOp) {
g.operations = append(g.operations, op)
}
-func NewDispatcher(feeder Feeder, workers int) *Dispatcher {
- if workers < 1 {
- workers = 1
- }
+func NewDispatcher(feeder Feeder, throttler Throttler, breaker CircuitBreaker) *Dispatcher {
d := &Dispatcher{
- workers: workers,
- feeder: feeder,
- inflight: make(map[string]*documentGroup),
+ feeder: feeder,
+ throttler: throttler,
+ circuitBreaker: breaker,
+ inflight: make(map[string]*documentGroup),
}
d.start()
return d
}
-func (d *Dispatcher) dispatchAll(g *documentGroup) int {
+func (d *Dispatcher) dispatchAll(g *documentGroup) {
g.mu.Lock()
defer g.mu.Unlock()
- failCount := len(g.operations)
- for i := 0; !g.failed && i < len(g.operations); i++ {
+ for i := 0; i < len(g.operations); i++ {
op := g.operations[i]
ok := false
- for op.attempts <= maxAttempts && !ok {
- op.attempts += 1
- // TODO(mpolden): Extract function which does throttling/circuit-breaking
+ for !ok {
+ op.attempts++
result := d.feeder.Send(op.document)
+ d.results <- result
ok = result.Status.Success()
+ if !d.shouldRetry(op, result) {
+ break
+ }
}
- if ok {
- failCount--
- } else {
- g.failed = true
- failCount = len(g.operations) - i
- }
+ d.releaseSlot()
}
g.operations = nil
- return failCount
+}
+
+func (d *Dispatcher) shouldRetry(op documentOp, result Result) bool {
+ if result.HTTPStatus/100 == 2 || result.HTTPStatus == 404 || result.HTTPStatus == 412 {
+ d.throttler.Success()
+ d.circuitBreaker.Success()
+ return false
+ }
+ if result.HTTPStatus == 429 || result.HTTPStatus == 503 {
+ d.throttler.Throttled(atomic.LoadInt64(&d.inflightCount))
+ return true
+ }
+ if result.HTTPStatus == 500 || result.HTTPStatus == 502 || result.HTTPStatus == 504 {
+ d.circuitBreaker.Error(fmt.Errorf("request failed with status %d", result.HTTPStatus))
+ if op.attempts <= maxAttempts {
+ return true
+ }
+ }
+ return false
}
func (d *Dispatcher) start() {
d.mu.Lock()
defer d.mu.Unlock()
+ d.ready = make(chan Id, 4096)
+ d.results = make(chan Result, 4096)
d.closed = false
- d.ready = make(chan Id, 4*d.workers)
- for i := 0; i < d.workers; i++ {
- d.wg.Add(1)
- go func() {
- defer d.wg.Done()
- for id := range d.ready {
- d.mu.RLock()
- group := d.inflight[id.String()]
- d.mu.RUnlock()
- if group != nil {
- failedDocs := d.dispatchAll(group)
- d.feeder.AddStats(Stats{Errors: int64(failedDocs)})
- }
- }
- }()
+ d.wg.Add(1)
+ go func() {
+ defer d.wg.Done()
+ d.readDocuments()
+ }()
+ d.resultWg.Add(1)
+ go func() {
+ defer d.resultWg.Done()
+ d.readResults()
+ }()
+}
+
+func (d *Dispatcher) readDocuments() {
+ for id := range d.ready {
+ d.mu.RLock()
+ group := d.inflight[id.String()]
+ d.mu.RUnlock()
+ if group != nil {
+ d.wg.Add(1)
+ go func() {
+ defer d.wg.Done()
+ d.dispatchAll(group)
+ }()
+ }
+ }
+}
+
+func (d *Dispatcher) readResults() {
+ for result := range d.results {
+ d.stats.Add(result.Stats)
}
}
func (d *Dispatcher) Enqueue(doc Document) error {
d.mu.Lock()
- defer d.mu.Unlock()
if d.closed {
return fmt.Errorf("dispatcher is closed")
}
@@ -112,18 +149,43 @@ func (d *Dispatcher) Enqueue(doc Document) error {
}
d.inflight[doc.Id.String()] = group
}
- d.ready <- doc.Id
+ d.mu.Unlock()
+ d.enqueueWithSlot(doc.Id)
return nil
}
-// Close closes the dispatcher and waits for all inflight operations to complete.
-func (d *Dispatcher) Close() error {
+func (d *Dispatcher) Stats() Stats { return d.stats }
+
+func (d *Dispatcher) enqueueWithSlot(id Id) {
+ d.acquireSlot()
+ d.ready <- id
+ d.throttler.Sent()
+}
+
+func (d *Dispatcher) acquireSlot() {
+ for atomic.LoadInt64(&d.inflightCount) >= d.throttler.TargetInflight() {
+ time.Sleep(time.Millisecond)
+ }
+ atomic.AddInt64(&d.inflightCount, 1)
+}
+
+func (d *Dispatcher) releaseSlot() { atomic.AddInt64(&d.inflightCount, -1) }
+
+func closeAndWait[T any](ch chan T, wg *sync.WaitGroup, d *Dispatcher, markClosed bool) {
d.mu.Lock()
if !d.closed {
- close(d.ready)
- d.closed = true
+ close(ch)
+ if markClosed {
+ d.closed = true
+ }
}
d.mu.Unlock()
- d.wg.Wait()
+ wg.Wait()
+}
+
+// Close closes the dispatcher and waits for all inflight operations to complete.
+func (d *Dispatcher) Close() error {
+ closeAndWait(d.ready, &d.wg, d, false)
+ closeAndWait(d.results, &d.resultWg, d, true)
return nil
}
diff --git a/client/go/internal/vespa/document/dispatcher_test.go b/client/go/internal/vespa/document/dispatcher_test.go
index 04e0928f2a3..8a6d8c6117c 100644
--- a/client/go/internal/vespa/document/dispatcher_test.go
+++ b/client/go/internal/vespa/document/dispatcher_test.go
@@ -3,6 +3,7 @@ package document
import (
"sync"
"testing"
+ "time"
"github.com/stretchr/testify/assert"
)
@@ -10,7 +11,6 @@ import (
type mockFeeder struct {
failAfterNDocs int
documents []Document
- stats Stats
mu sync.Mutex
}
@@ -23,24 +23,24 @@ func (f *mockFeeder) failAfterN(docs int) {
func (f *mockFeeder) Send(doc Document) Result {
f.mu.Lock()
defer f.mu.Unlock()
+ result := Result{Id: doc.Id}
if f.failAfterNDocs > 0 && len(f.documents) >= f.failAfterNDocs {
- return Result{Id: doc.Id, Status: StatusVespaFailure}
+ result.Status = StatusVespaFailure
+ } else {
+ f.documents = append(f.documents, doc)
}
- f.documents = append(f.documents, doc)
- return Result{Id: doc.Id}
-}
-
-func (f *mockFeeder) Stats() Stats { return f.stats }
-
-func (f *mockFeeder) AddStats(stats Stats) {
- f.mu.Lock()
- defer f.mu.Unlock()
- f.stats.Add(stats)
+ if !result.Status.Success() {
+ result.Stats.Errors = 1
+ }
+ return result
}
func TestDispatcher(t *testing.T) {
feeder := &mockFeeder{}
- dispatcher := NewDispatcher(feeder, 2)
+ clock := &manualClock{tick: time.Second}
+ throttler := newThrottler(clock.now)
+ breaker := NewCircuitBreaker(time.Second, 0)
+ dispatcher := NewDispatcher(feeder, throttler, breaker)
docs := []Document{
{Id: mustParseId("id:ns:type::doc1"), Operation: OperationPut, Body: []byte(`{"fields":{"foo": "123"}}`)},
{Id: mustParseId("id:ns:type::doc2"), Operation: OperationPut, Body: []byte(`{"fields":{"bar": "456"}}`)},
@@ -70,7 +70,10 @@ func TestDispatcherOrdering(t *testing.T) {
{Id: mustParseId("id:ns:type::doc8"), Operation: OperationPut},
{Id: mustParseId("id:ns:type::doc9"), Operation: OperationPut},
}
- dispatcher := NewDispatcher(feeder, len(docs))
+ clock := &manualClock{tick: time.Second}
+ throttler := newThrottler(clock.now)
+ breaker := NewCircuitBreaker(time.Second, 0)
+ dispatcher := NewDispatcher(feeder, throttler, breaker)
for _, d := range docs {
dispatcher.Enqueue(d)
}
@@ -90,7 +93,7 @@ func TestDispatcherOrdering(t *testing.T) {
}
assert.Equal(t, len(docs), len(feeder.documents))
assert.Equal(t, wantDocs, gotDocs)
- assert.Equal(t, int64(0), feeder.Stats().Errors)
+ assert.Equal(t, int64(0), dispatcher.Stats().Errors)
}
func TestDispatcherOrderingWithFailures(t *testing.T) {
@@ -103,26 +106,26 @@ func TestDispatcherOrderingWithFailures(t *testing.T) {
{Id: commonId, Operation: OperationRemove}, // fails
}
feeder.failAfterN(2)
- dispatcher := NewDispatcher(feeder, len(docs))
+ clock := &manualClock{tick: time.Second}
+ throttler := newThrottler(clock.now)
+ breaker := NewCircuitBreaker(time.Second, 0)
+ dispatcher := NewDispatcher(feeder, throttler, breaker)
for _, d := range docs {
dispatcher.Enqueue(d)
}
dispatcher.Close()
wantDocs := docs[:2]
assert.Equal(t, wantDocs, feeder.documents)
- assert.Equal(t, int64(2), feeder.Stats().Errors)
+ assert.Equal(t, int64(2), dispatcher.Stats().Errors)
- // Dispatching more documents for same ID fails implicitly
+ // Dispatching more documents for same ID succeed
feeder.failAfterN(0)
dispatcher.start()
dispatcher.Enqueue(Document{Id: commonId, Operation: OperationPut})
dispatcher.Enqueue(Document{Id: commonId, Operation: OperationRemove})
- // Other IDs are fine
- doc2 := Document{Id: mustParseId("id:ns:type::doc2"), Operation: OperationPut}
- doc3 := Document{Id: mustParseId("id:ns:type::doc3"), Operation: OperationPut}
- dispatcher.Enqueue(doc2)
- dispatcher.Enqueue(doc3)
+ dispatcher.Enqueue(Document{Id: mustParseId("id:ns:type::doc2"), Operation: OperationPut})
+ dispatcher.Enqueue(Document{Id: mustParseId("id:ns:type::doc3"), Operation: OperationPut})
dispatcher.Close()
- assert.Equal(t, int64(4), feeder.Stats().Errors)
- assert.Equal(t, 4, len(feeder.documents))
+ assert.Equal(t, int64(2), dispatcher.Stats().Errors)
+ assert.Equal(t, 6, len(feeder.documents))
}
diff --git a/client/go/internal/vespa/document/feeder.go b/client/go/internal/vespa/document/feeder.go
index 6996e649d24..8bdd5bca5ba 100644
--- a/client/go/internal/vespa/document/feeder.go
+++ b/client/go/internal/vespa/document/feeder.go
@@ -23,17 +23,19 @@ const (
// Result represents the result of a feeding operation.
type Result struct {
- Id Id
- Status Status
- Message string
- Trace string
- Err error
+ Id Id
+ Status Status
+ HTTPStatus int
+ Message string
+ Trace string
+ Err error
+ Stats Stats
}
// Success returns whether status s is considered a success.
func (s Status) Success() bool { return s == StatusSuccess || s == StatusConditionNotMet }
-// Stats represents the summed statistics of a feeder.
+// Stats represents feeding operation statistics.
type Stats struct {
Requests int64
Responses int64
@@ -47,8 +49,6 @@ type Stats struct {
BytesRecv int64
}
-func NewStats() Stats { return Stats{ResponsesByCode: make(map[int]int64)} }
-
// AvgLatency returns the average latency for a request.
func (s Stats) AvgLatency() time.Duration {
requests := s.Requests
@@ -69,6 +69,9 @@ func (s Stats) Successes() int64 {
func (s *Stats) Add(other Stats) {
s.Requests += other.Requests
s.Responses += other.Responses
+ if s.ResponsesByCode == nil && other.ResponsesByCode != nil {
+ s.ResponsesByCode = make(map[int]int64)
+ }
for code, count := range other.ResponsesByCode {
_, ok := s.ResponsesByCode[code]
if ok {
@@ -91,8 +94,4 @@ func (s *Stats) Add(other Stats) {
}
// Feeder is the interface for a consumer of documents.
-type Feeder interface {
- Send(Document) Result
- Stats() Stats
- AddStats(Stats)
-}
+type Feeder interface{ Send(Document) Result }
diff --git a/client/go/internal/vespa/document/feeder_test.go b/client/go/internal/vespa/document/feeder_test.go
index 1368d871436..a7d92495889 100644
--- a/client/go/internal/vespa/document/feeder_test.go
+++ b/client/go/internal/vespa/document/feeder_test.go
@@ -7,7 +7,7 @@ import (
)
func TestStatsAdd(t *testing.T) {
- got := NewStats()
+ var got Stats
got.Add(Stats{Requests: 1})
got.Add(Stats{Requests: 1})
got.Add(Stats{Responses: 1})
diff --git a/client/go/internal/vespa/document/http.go b/client/go/internal/vespa/document/http.go
index 2e01d4564ab..4dadcd1d05c 100644
--- a/client/go/internal/vespa/document/http.go
+++ b/client/go/internal/vespa/document/http.go
@@ -9,7 +9,6 @@ import (
"net/url"
"strconv"
"strings"
- "sync"
"time"
"github.com/vespa-engine/vespa/client/go/internal/util"
@@ -19,8 +18,6 @@ import (
type Client struct {
options ClientOptions
httpClient util.HTTPClient
- stats Stats
- mu sync.Mutex
now func() time.Time
}
@@ -47,7 +44,6 @@ func NewClient(options ClientOptions, httpClient util.HTTPClient) *Client {
c := &Client{
options: options,
httpClient: httpClient,
- stats: NewStats(),
now: time.Now,
}
return c
@@ -116,49 +112,42 @@ func (c *Client) feedURL(d Document, queryParams url.Values) (string, *url.URL,
// Send given document the URL configured in this client.
func (c *Client) Send(document Document) Result {
start := c.now()
- stats := NewStats()
- stats.Requests = 1
- defer func() {
- latency := c.now().Sub(start)
- stats.TotalLatency = latency
- stats.MinLatency = latency
- stats.MaxLatency = latency
- c.AddStats(stats)
- }()
+ result := Result{Id: document.Id}
+ result.Stats.Requests = 1
method, url, err := c.feedURL(document, c.queryParams())
if err != nil {
- stats.Errors = 1
- return Result{Status: StatusError, Err: err}
+ result.Stats.Errors = 1
+ result.Err = err
+ return result
}
req, err := http.NewRequest(method, url.String(), bytes.NewReader(document.Body))
if err != nil {
- stats.Errors = 1
- return Result{Status: StatusError, Err: err}
+ result.Stats.Errors = 1
+ result.Status = StatusError
+ result.Err = err
+ return result
}
- resp, err := c.httpClient.Do(req, c.options.Timeout)
+ resp, err := c.httpClient.Do(req, 190*time.Second)
if err != nil {
- stats.Errors = 1
- return Result{Status: StatusTransportFailure, Err: err}
+ result.Stats.Errors = 1
+ result.Status = StatusTransportFailure
+ result.Err = err
+ return result
}
defer resp.Body.Close()
- stats.Responses = 1
- stats.ResponsesByCode = map[int]int64{
+ result.Stats.Responses = 1
+ result.Stats.ResponsesByCode = map[int]int64{
resp.StatusCode: 1,
}
- stats.BytesSent = int64(len(document.Body))
- return c.createResult(document.Id, &stats, resp)
-}
-
-func (c *Client) Stats() Stats { return c.stats }
-
-func (c *Client) AddStats(stats Stats) {
- c.mu.Lock()
- defer c.mu.Unlock()
- c.stats.Add(stats)
+ result.Stats.BytesSent = int64(len(document.Body))
+ elapsed := c.now().Sub(start)
+ result.Stats.TotalLatency = elapsed
+ result.Stats.MinLatency = elapsed
+ result.Stats.MaxLatency = elapsed
+ return c.resultWithResponse(resp, result)
}
-func (c *Client) createResult(id Id, stats *Stats, resp *http.Response) Result {
- result := Result{Id: id}
+func (c *Client) resultWithResponse(resp *http.Response, result Result) Result {
switch resp.StatusCode {
case 200:
result.Status = StatusSuccess
@@ -181,9 +170,9 @@ func (c *Client) createResult(id Id, stats *Stats, resp *http.Response) Result {
}
result.Message = body.Message
result.Trace = string(body.Trace)
- stats.BytesRecv = cr.bytesRead
+ result.Stats.BytesRecv = cr.bytesRead
if !result.Status.Success() {
- stats.Errors = 1
+ result.Stats.Errors = 1
}
return result
}
diff --git a/client/go/internal/vespa/document/http_test.go b/client/go/internal/vespa/document/http_test.go
index f02c87730d5..311668fa16e 100644
--- a/client/go/internal/vespa/document/http_test.go
+++ b/client/go/internal/vespa/document/http_test.go
@@ -19,11 +19,12 @@ type manualClock struct {
}
func (c *manualClock) now() time.Time {
- t := c.t
- c.t = c.t.Add(c.tick)
- return t
+ c.advance(c.tick)
+ return c.t
}
+func (c *manualClock) advance(d time.Duration) { c.t = c.t.Add(d) }
+
func TestClientSend(t *testing.T) {
docs := []Document{
{Create: true, Id: mustParseId("id:ns:type::doc1"), Operation: OperationUpdate, Body: []byte(`{"fields":{"foo": "123"}}`)},
@@ -37,6 +38,7 @@ func TestClientSend(t *testing.T) {
}, &httpClient)
clock := manualClock{t: time.Now(), tick: time.Second}
client.now = clock.now
+ var stats Stats
for i, doc := range docs {
if i < 2 {
httpClient.NextResponseString(200, `{"message":"All good!"}`)
@@ -44,6 +46,7 @@ func TestClientSend(t *testing.T) {
httpClient.NextResponseString(502, `{"message":"Good bye, cruel world!"}`)
}
res := client.Send(doc)
+ stats.Add(res.Stats)
if res.Err != nil {
t.Fatalf("got unexpected error %q", res.Err)
}
@@ -64,7 +67,6 @@ func TestClientSend(t *testing.T) {
t.Errorf("got r.Body = %q, want %q", string(body), string(wantBody))
}
}
- stats := client.Stats()
want := Stats{
Requests: 3,
Responses: 3,
diff --git a/client/go/internal/vespa/document/throttler.go b/client/go/internal/vespa/document/throttler.go
new file mode 100644
index 00000000000..69bb7c8d7ac
--- /dev/null
+++ b/client/go/internal/vespa/document/throttler.go
@@ -0,0 +1,117 @@
+package document
+
+import (
+ "math"
+ "math/rand"
+ "sync/atomic"
+ "time"
+)
+
+const (
+ throttlerWeight = 0.7
+ // TODO(mpolden): Multiply this by connections per endpoint, and number of endpoints when this becomes configurable
+ // for local target
+ throttlerMinInflight = 16
+ throttlerMaxInflight = 256 * throttlerMinInflight // 4096 max streams per connection on the server side
+)
+
+type Throttler interface {
+ // Sent notifies the the throttler that a document has been sent.
+ Sent()
+ // Success notifies the throttler that document operation succeeded.
+ Success()
+ // Throttled notifies the throttler that a throttling event occured while count documents were in-flight.
+ Throttled(count int64)
+ // TargetInflight returns the ideal number of documents to have in-flight now.
+ TargetInflight() int64
+}
+
+type dynamicThrottler struct {
+ ok int64
+ targetInflight int64
+ targetTimesTen int64
+
+ throughputs []float64
+ sent int64
+
+ start time.Time
+ now func() time.Time
+}
+
+func newThrottler(nowFunc func() time.Time) *dynamicThrottler {
+ return &dynamicThrottler{
+ throughputs: make([]float64, 128),
+ start: nowFunc(),
+ now: nowFunc,
+
+ targetInflight: 8 * throttlerMinInflight,
+ targetTimesTen: 10 * throttlerMaxInflight,
+ }
+}
+
+func NewThrottler() Throttler { return newThrottler(time.Now) }
+
+func (t *dynamicThrottler) Sent() {
+ currentInflight := atomic.LoadInt64(&t.targetInflight)
+ t.sent++
+ if t.sent*t.sent*t.sent < 100*currentInflight*currentInflight {
+ return
+ }
+ t.sent = 0
+ now := t.now()
+ elapsed := now.Sub(t.start)
+ t.start = now
+ currentThroughput := float64(atomic.SwapInt64(&t.ok, 0)) / float64(elapsed)
+
+ // Use buckets for throughput over inflight, along the log-scale, in [minInflight, maxInflight).
+ index := int(float64(len(t.throughputs)) * math.Log(max(1, min(255, float64(currentInflight)/throttlerMinInflight))) / math.Log(256))
+ t.throughputs[index] = currentThroughput
+
+ // Loop over throughput measurements and pick the one which optimises throughput and latency.
+ choice := float64(currentInflight)
+ maxObjective := float64(-1)
+ for i := len(t.throughputs) - 1; i >= 0; i-- {
+ if t.throughputs[i] == 0 {
+ continue // Skip unknown values
+ }
+ inflight := float64(throttlerMinInflight) * math.Pow(256, (float64(i)+0.5)/float64(len(t.throughputs)))
+ objective := t.throughputs[i] * math.Pow(inflight, throttlerWeight-1) // Optimise throughput (weight), but also latency (1 - weight)
+ if objective > maxObjective {
+ maxObjective = objective
+ choice = inflight
+ }
+ }
+ target := int64((rand.Float64()*0.20 + 0.92) * choice) // Random walk, skewed towards increase
+ atomic.StoreInt64(&t.targetInflight, max(throttlerMinInflight, min(throttlerMaxInflight, target)))
+}
+
+func (t *dynamicThrottler) Success() {
+ atomic.AddInt64(&t.targetTimesTen, 1)
+ atomic.AddInt64(&t.ok, 1)
+}
+
+func (t *dynamicThrottler) Throttled(inflight int64) {
+ atomic.StoreInt64(&t.targetTimesTen, max(inflight*5, throttlerMinInflight*10))
+}
+
+func (t *dynamicThrottler) TargetInflight() int64 {
+ staticTargetInflight := min(throttlerMaxInflight, atomic.LoadInt64(&t.targetTimesTen)/10)
+ targetInflight := atomic.LoadInt64(&t.targetInflight)
+ return min(staticTargetInflight, targetInflight)
+}
+
+type number interface{ float64 | int64 }
+
+func min[T number](x, y T) T {
+ if x < y {
+ return x
+ }
+ return y
+}
+
+func max[T number](x, y T) T {
+ if x > y {
+ return x
+ }
+ return y
+}
diff --git a/client/go/internal/vespa/document/throttler_test.go b/client/go/internal/vespa/document/throttler_test.go
new file mode 100644
index 00000000000..2fd1e73a45a
--- /dev/null
+++ b/client/go/internal/vespa/document/throttler_test.go
@@ -0,0 +1,21 @@
+package document
+
+import (
+ "testing"
+ "time"
+)
+
+func TestThrottler(t *testing.T) {
+ clock := &manualClock{tick: time.Second}
+ tr := newThrottler(clock.now)
+ for i := 0; i < 100; i++ {
+ tr.Sent()
+ }
+ if got, want := tr.TargetInflight(), int64(128); got != want {
+ t.Errorf("got TargetInflight() = %d, but want %d", got, want)
+ }
+ tr.Throttled(5)
+ if got, want := tr.TargetInflight(), int64(16); got != want {
+ t.Errorf("got TargetInflight() = %d, but want %d", got, want)
+ }
+}
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/ContentCluster.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/ContentCluster.java
index 9347fadc0e0..9538167c6de 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/ContentCluster.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/ContentCluster.java
@@ -136,15 +136,9 @@ public class ContentCluster {
*/
public NodeStateChangeChecker.Result calculateEffectOfNewState(
Node node, ClusterState clusterState, SetUnitStateRequest.Condition condition,
- NodeState oldState, NodeState newState, boolean inMoratorium, int maxNumberOfGroupsAllowedToBeDown) {
-
- NodeStateChangeChecker nodeStateChangeChecker = new NodeStateChangeChecker(
- distribution.getRedundancy(),
- new HierarchicalGroupVisitingAdapter(distribution),
- clusterInfo,
- inMoratorium,
- maxNumberOfGroupsAllowedToBeDown
- );
+ NodeState oldState, NodeState newState, boolean inMoratorium) {
+
+ NodeStateChangeChecker nodeStateChangeChecker = new NodeStateChangeChecker(this, inMoratorium);
return nodeStateChangeChecker.evaluateTransition(node, clusterState, condition, oldState, newState);
}
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeChecker.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeChecker.java
index 7a77bb2b571..2025dfef562 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeChecker.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeChecker.java
@@ -42,17 +42,14 @@ public class NodeStateChangeChecker {
private final HierarchicalGroupVisiting groupVisiting;
private final ClusterInfo clusterInfo;
private final boolean inMoratorium;
+ private final int maxNumberOfGroupsAllowedToBeDown;
- public NodeStateChangeChecker(
- int requiredRedundancy,
- HierarchicalGroupVisiting groupVisiting,
- ClusterInfo clusterInfo,
- boolean inMoratorium,
- int maxNumberOfGroupsAllowedToBeDown) {
- this.requiredRedundancy = requiredRedundancy;
- this.groupVisiting = groupVisiting;
- this.clusterInfo = clusterInfo;
+ public NodeStateChangeChecker(ContentCluster cluster, boolean inMoratorium) {
+ this.requiredRedundancy = cluster.getDistribution().getRedundancy();
+ this.groupVisiting = new HierarchicalGroupVisitingAdapter(cluster.getDistribution());
+ this.clusterInfo = cluster.clusterInfo();
this.inMoratorium = inMoratorium;
+ this.maxNumberOfGroupsAllowedToBeDown = cluster.maxNumberOfGroupsAllowedToBeDown();
}
public static class Result {
@@ -392,8 +389,7 @@ public class NodeStateChangeChecker {
return allowSettingOfWantedState();
}
- private Result checkStorageNodesForDistributor(
- DistributorNodeInfo distributorNodeInfo, List<StorageNode> storageNodes, Node node) {
+ private Result checkStorageNodesForDistributor(DistributorNodeInfo distributorNodeInfo, List<StorageNode> storageNodes, Node node) {
for (StorageNode storageNode : storageNodes) {
if (storageNode.getIndex() == node.getIndex()) {
Integer minReplication = storageNode.getMinCurrentReplicationFactorOrNull();
diff --git a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequest.java b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequest.java
index a2e77b4e3dd..1c72594377a 100644
--- a/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequest.java
+++ b/clustercontroller-core/src/main/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequest.java
@@ -126,8 +126,8 @@ public class SetNodeStateRequest extends Request<SetResponse> {
NodeState wantedState = nodeInfo.getUserWantedState();
NodeState newWantedState = getRequestedNodeState(newStates, node);
- Result result = cluster.calculateEffectOfNewState(node, currentClusterState, condition, wantedState, newWantedState,
- inMasterMoratorium, cluster.maxNumberOfGroupsAllowedToBeDown());
+ Result result = cluster.calculateEffectOfNewState(node, currentClusterState, condition, wantedState,
+ newWantedState, inMasterMoratorium);
log.log(Level.FINE, () -> "node=" + node +
" current-cluster-state=" + currentClusterState + // Includes version in output format
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeCheckerTest.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeCheckerTest.java
index 60d4866a33e..e789a3cc6a6 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeCheckerTest.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/NodeStateChangeCheckerTest.java
@@ -3,8 +3,6 @@ package com.yahoo.vespa.clustercontroller.core;
import com.yahoo.vdslib.distribution.ConfiguredNode;
import com.yahoo.vdslib.distribution.Distribution;
-import com.yahoo.vdslib.distribution.Group;
-import com.yahoo.vdslib.distribution.GroupVisitor;
import com.yahoo.vdslib.state.ClusterState;
import com.yahoo.vdslib.state.Node;
import com.yahoo.vdslib.state.NodeState;
@@ -22,16 +20,13 @@ import static com.yahoo.vdslib.state.NodeType.STORAGE;
import static com.yahoo.vdslib.state.State.DOWN;
import static com.yahoo.vdslib.state.State.INITIALIZING;
import static com.yahoo.vdslib.state.State.UP;
+import static com.yahoo.vespa.clustercontroller.core.NodeStateChangeChecker.Result;
import static com.yahoo.vespa.clustercontroller.utils.staterestapi.requests.SetUnitStateRequest.Condition.FORCE;
import static com.yahoo.vespa.clustercontroller.utils.staterestapi.requests.SetUnitStateRequest.Condition.SAFE;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import static com.yahoo.vespa.clustercontroller.core.NodeStateChangeChecker.Result;
public class NodeStateChangeCheckerTest {
@@ -45,11 +40,6 @@ public class NodeStateChangeCheckerTest {
private static final NodeState MAINTENANCE_NODE_STATE = createNodeState(State.MAINTENANCE, "Orchestrator");
private static final NodeState DOWN_NODE_STATE = createNodeState(DOWN, "RetireEarlyExpirer");
- private static final HierarchicalGroupVisiting noopVisiting = new HierarchicalGroupVisiting() {
- @Override public boolean isHierarchical() { return false; }
- @Override public void visit(GroupVisitor visitor) { }
- };
-
private static NodeState createNodeState(State state, String description) {
return new NodeState(STORAGE, state).setDescription(description);
}
@@ -67,15 +57,16 @@ public class NodeStateChangeCheckerTest {
}
private NodeStateChangeChecker createChangeChecker(ContentCluster cluster) {
- return new NodeStateChangeChecker(requiredRedundancy, noopVisiting, cluster.clusterInfo(),
- false, cluster.maxNumberOfGroupsAllowedToBeDown());
+ return new NodeStateChangeChecker(cluster, false);
}
private ContentCluster createCluster(int nodeCount) {
+ return createCluster(nodeCount, 1);
+ }
+
+ private ContentCluster createCluster(int nodeCount, int groupCount) {
Collection<ConfiguredNode> nodes = createNodes(nodeCount);
- Distribution distribution = mock(Distribution.class);
- Group group = new Group(2, "two");
- when(distribution.getRootGroup()).thenReturn(group);
+ Distribution distribution = new Distribution(createDistributionConfig(nodeCount, groupCount));
return new ContentCluster("Clustername", nodes, distribution);
}
@@ -128,8 +119,7 @@ public class NodeStateChangeCheckerTest {
@Test
void testDeniedInMoratorium() {
ContentCluster cluster = createCluster(4);
- var nodeStateChangeChecker = new NodeStateChangeChecker(
- requiredRedundancy, noopVisiting, cluster.clusterInfo(), true, cluster.maxNumberOfGroupsAllowedToBeDown());
+ var nodeStateChangeChecker = new NodeStateChangeChecker(cluster, true);
Result result = nodeStateChangeChecker.evaluateTransition(
new Node(STORAGE, 10), defaultAllUpClusterState(), SAFE,
UP_NODE_STATE, MAINTENANCE_NODE_STATE);
@@ -171,7 +161,7 @@ public class NodeStateChangeCheckerTest {
@Test
void testSafeMaintenanceDisallowedWhenOtherDistributorInFlatClusterIsSuspended() {
- // Nodes 0-3, storage node 0 being in maintenance with "Orchestrator" description.
+ // Nodes 0-3, distributor 0 being down with "Orchestrator" description.
ContentCluster cluster = createCluster(4);
cluster.clusterInfo().getDistributorNodeInfo(0)
.setWantedState(new NodeState(DISTRIBUTOR, DOWN).setDescription("Orchestrator"));
@@ -193,12 +183,10 @@ public class NodeStateChangeCheckerTest {
void testSafeMaintenanceDisallowedWhenDistributorInGroupIsDown() {
// Nodes 0-3, distributor 0 being in maintenance with "Orchestrator" description.
// 2 groups: nodes 0-1 is group 0, 2-3 is group 1.
- ContentCluster cluster = createCluster(4);
+ ContentCluster cluster = createCluster(4, 2);
cluster.clusterInfo().getDistributorNodeInfo(0)
.setWantedState(new NodeState(STORAGE, DOWN).setDescription("Orchestrator"));
- HierarchicalGroupVisiting visiting = makeHierarchicalGroupVisitingWith2Groups(4);
- var nodeStateChangeChecker = new NodeStateChangeChecker(
- requiredRedundancy, visiting, cluster.clusterInfo(), false, cluster.maxNumberOfGroupsAllowedToBeDown());
+ var nodeStateChangeChecker = new NodeStateChangeChecker(cluster, false);
ClusterState clusterStateWith0InMaintenance = clusterState(String.format(
"version:%d distributor:4 .0.s:d storage:4",
currentClusterStateVersion));
@@ -228,11 +216,9 @@ public class NodeStateChangeCheckerTest {
void testSafeMaintenanceWhenOtherStorageNodeInGroupIsSuspended() {
// Nodes 0-3, storage node 0 being in maintenance with "Orchestrator" description.
// 2 groups: nodes 0-1 is group 0, 2-3 is group 1.
- ContentCluster cluster = createCluster(4);
+ ContentCluster cluster = createCluster(4, 2);
cluster.clusterInfo().getStorageNodeInfo(0).setWantedState(new NodeState(STORAGE, State.MAINTENANCE).setDescription("Orchestrator"));
- HierarchicalGroupVisiting visiting = makeHierarchicalGroupVisitingWith2Groups(4);
- var nodeStateChangeChecker = new NodeStateChangeChecker(
- requiredRedundancy, visiting, cluster.clusterInfo(), false, cluster.maxNumberOfGroupsAllowedToBeDown());
+ var nodeStateChangeChecker = new NodeStateChangeChecker(cluster, false);
ClusterState clusterStateWith0InMaintenance = clusterState(String.format(
"version:%d distributor:4 storage:4 .0.s:m",
currentClusterStateVersion));
@@ -259,46 +245,6 @@ public class NodeStateChangeCheckerTest {
}
}
- /**
- * Make a HierarchicalGroupVisiting with the given number of nodes, with 2 groups:
- * Group "0" is nodes 0-1, group "1" is 2-3.
- */
- private HierarchicalGroupVisiting makeHierarchicalGroupVisitingWith2Groups(int nodes) {
- int groups = 2;
- if (nodes % groups != 0) {
- throw new IllegalArgumentException("Cannot have 2 groups with an odd number of nodes: " + nodes);
- }
- int nodesPerGroup = nodes / groups;
-
- var configBuilder = new StorDistributionConfig.Builder()
- .active_per_leaf_group(true)
- .ready_copies(2)
- .redundancy(2)
- .initial_redundancy(2);
-
- configBuilder.group(new StorDistributionConfig.Group.Builder()
- .index("invalid")
- .name("invalid")
- .capacity(nodes)
- .partitions("1|*"));
-
- int nodeIndex = 0;
- for (int i = 0; i < groups; ++i) {
- var groupBuilder = new StorDistributionConfig.Group.Builder()
- .index(String.valueOf(i))
- .name(String.valueOf(i))
- .capacity(nodesPerGroup)
- .partitions("");
- for (int j = 0; j < nodesPerGroup; ++j, ++nodeIndex) {
- groupBuilder.nodes(new StorDistributionConfig.Group.Nodes.Builder()
- .index(nodeIndex));
- }
- configBuilder.group(groupBuilder);
- }
-
- return new HierarchicalGroupVisitingAdapter(new Distribution(configBuilder.build()));
- }
-
@Test
void testSafeSetStateDistributors() {
NodeStateChangeChecker nodeStateChangeChecker = createChangeChecker(createCluster(1));
@@ -753,4 +699,61 @@ public class NodeStateChangeCheckerTest {
nodes.add(new ConfiguredNode(i, false));
return nodes;
}
+
+ private StorDistributionConfig createDistributionConfig(int nodes) {
+ var configBuilder = new StorDistributionConfig.Builder()
+ .ready_copies(requiredRedundancy)
+ .redundancy(requiredRedundancy)
+ .initial_redundancy(requiredRedundancy);
+
+ var groupBuilder = new StorDistributionConfig.Group.Builder()
+ .index("invalid")
+ .name("invalid")
+ .capacity(nodes);
+ int nodeIndex = 0;
+ for (int j = 0; j < nodes; ++j, ++nodeIndex) {
+ groupBuilder.nodes(new StorDistributionConfig.Group.Nodes.Builder()
+ .index(nodeIndex));
+ }
+ configBuilder.group(groupBuilder);
+
+ return configBuilder.build();
+ }
+
+ // When more than 1 group
+ private StorDistributionConfig createDistributionConfig(int nodes, int groups) {
+ if (groups == 1) return createDistributionConfig(nodes);
+
+ if (nodes % groups != 0)
+ throw new IllegalArgumentException("Cannot have " + groups + " groups with an odd number of nodes: " + nodes);
+
+ int nodesPerGroup = nodes / groups;
+
+ var configBuilder = new StorDistributionConfig.Builder()
+ .active_per_leaf_group(true)
+ .ready_copies(groups)
+ .redundancy(groups)
+ .initial_redundancy(groups);
+
+ configBuilder.group(new StorDistributionConfig.Group.Builder()
+ .index("invalid")
+ .name("invalid")
+ .capacity(nodes)
+ .partitions("1|*"));
+
+ for (int i = 0; i < groups; ++i) {
+ var groupBuilder = new StorDistributionConfig.Group.Builder()
+ .index(String.valueOf(i))
+ .name(String.valueOf(i))
+ .capacity(nodesPerGroup)
+ .partitions("");
+ for (int nodeIndex = 0; nodeIndex < nodesPerGroup; ++nodeIndex) {
+ groupBuilder.nodes(new StorDistributionConfig.Group.Nodes.Builder()
+ .index(nodeIndex));
+ }
+ configBuilder.group(groupBuilder);
+ }
+ return configBuilder.build();
+ }
+
}
diff --git a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequestTest.java b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequestTest.java
index b208ff7fb27..6d93eadfe2a 100644
--- a/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequestTest.java
+++ b/clustercontroller-core/src/test/java/com/yahoo/vespa/clustercontroller/core/restapiv2/requests/SetNodeStateRequestTest.java
@@ -1,5 +1,4 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
package com.yahoo.vespa.clustercontroller.core.restapiv2.requests;
import com.yahoo.vdslib.state.ClusterState;
@@ -17,14 +16,12 @@ import com.yahoo.vespa.clustercontroller.utils.staterestapi.response.SetResponse
import com.yahoo.vespa.clustercontroller.utils.staterestapi.response.UnitState;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
-
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
-import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
@@ -32,7 +29,10 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class SetNodeStateRequestTest {
+
private static final String REASON = "operator";
+ private static final boolean inMasterMoratorium = false;
+
private final ContentCluster cluster = mock(ContentCluster.class);
private final SetUnitStateRequest.Condition condition = SetUnitStateRequest.Condition.SAFE;
private final Map<String, UnitState> newStates = new HashMap<>();
@@ -41,7 +41,7 @@ public class SetNodeStateRequestTest {
private final Node storageNode = new Node(NodeType.STORAGE, NODE_INDEX);
private final NodeListener stateListener = mock(NodeListener.class);
private final ClusterState currentClusterState = mock(ClusterState.class);
- private boolean inMasterMoratorium = false;
+
private boolean probe = false;
@BeforeEach
@@ -130,7 +130,7 @@ public class SetNodeStateRequestTest {
when(unitState.getId()).thenReturn(wantedStateString);
when(unitState.getReason()).thenReturn(REASON);
- when(cluster.calculateEffectOfNewState(any(), any(), any(), any(), any(), anyBoolean(), anyInt())).thenReturn(result);
+ when(cluster.calculateEffectOfNewState(any(), any(), any(), any(), any(), anyBoolean())).thenReturn(result);
when(storageNodeInfo.isStorage()).thenReturn(storageNode.getType() == NodeType.STORAGE);
when(storageNodeInfo.getNodeIndex()).thenReturn(storageNode.getIndex());
diff --git a/config-model-api/src/main/java/com/yahoo/config/model/api/ModelContext.java b/config-model-api/src/main/java/com/yahoo/config/model/api/ModelContext.java
index 21797f0f469..fef2354c452 100644
--- a/config-model-api/src/main/java/com/yahoo/config/model/api/ModelContext.java
+++ b/config-model-api/src/main/java/com/yahoo/config/model/api/ModelContext.java
@@ -72,7 +72,7 @@ public interface ModelContext {
* - Remove all flag data files from hosted-feature-flag repository
*/
interface FeatureFlags {
- @ModelFeatureFlag(owners = {"baldersheim"}, comment = "Revisit in May or June 2021") default double defaultTermwiseLimit() { throw new UnsupportedOperationException("TODO specify default value"); }
+ @ModelFeatureFlag(owners = {"baldersheim"}, comment = "Revisit in May or June 2023") default double defaultTermwiseLimit() { throw new UnsupportedOperationException("TODO specify default value"); }
@ModelFeatureFlag(owners = {"baldersheim"}, comment = "Select sequencer type use while feeding") default String feedSequencerType() { return "THROUGHPUT"; }
@ModelFeatureFlag(owners = {"baldersheim"}) default String responseSequencerType() { throw new UnsupportedOperationException("TODO specify default value"); }
@ModelFeatureFlag(owners = {"baldersheim"}) default String queryDispatchPolicy() { return "adaptive"; }
@@ -112,6 +112,7 @@ public interface ModelContext {
@ModelFeatureFlag(owners = {"arnej","baldersheim"}, removeAfter = "8.110") default boolean useOldJdiscContainerStartup() { return false; }
@ModelFeatureFlag(owners = {"tokle, bjorncs"}, removeAfter = "8.108") default boolean enableDataPlaneFilter() { return true; }
@ModelFeatureFlag(owners = {"arnej, bjorncs"}) default boolean enableGlobalPhase() { return true; }
+ @ModelFeatureFlag(owners = {"baldersheim"}, comment = "Select summary decode type") default String summaryDecodePolicy() { return "eager"; }
//Below are all flags that must be kept until 7 is out of the door
@ModelFeatureFlag(owners = {"arnej"}, removeAfter="7.last") default boolean ignoreThreadStackSizes() { return false; }
diff --git a/config-model/src/main/java/com/yahoo/config/model/deploy/TestProperties.java b/config-model/src/main/java/com/yahoo/config/model/deploy/TestProperties.java
index ecbb990f096..c72aa23a836 100644
--- a/config-model/src/main/java/com/yahoo/config/model/deploy/TestProperties.java
+++ b/config-model/src/main/java/com/yahoo/config/model/deploy/TestProperties.java
@@ -43,6 +43,7 @@ public class TestProperties implements ModelContext.Properties, ModelContext.Fea
private double defaultTermwiseLimit = 1.0;
private String jvmGCOptions = null;
private String queryDispatchPolicy = "adaptive";
+ private String summaryDecodePolicy = "eager";
private String sequencerType = "THROUGHPUT";
private boolean firstTimeDeployment = false;
private String responseSequencerType = "ADAPTIVE";
@@ -134,6 +135,7 @@ public class TestProperties implements ModelContext.Properties, ModelContext.Fea
@Override public int heapSizePercentage() { return heapSizePercentage; }
@Override public int rpcEventsBeforeWakeup() { return rpc_events_before_wakeup; }
@Override public String queryDispatchPolicy() { return queryDispatchPolicy; }
+ @Override public String summaryDecodePolicy() { return summaryDecodePolicy; }
@Override public boolean useRestrictedDataPlaneBindings() { return useRestrictedDataPlaneBindings; }
@Override public Optional<CloudAccount> cloudAccount() { return cloudAccount; }
@Override public boolean allowUserFilters() { return allowUserFilters; }
@@ -191,6 +193,10 @@ public class TestProperties implements ModelContext.Properties, ModelContext.Fea
queryDispatchPolicy = policy;
return this;
}
+ public TestProperties setSummaryDecodePolicy(String type) {
+ summaryDecodePolicy = type;
+ return this;
+ }
public TestProperties setFeedSequencerType(String type) {
sequencerType = type;
return this;
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java b/config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java
index 9c0a6b1b224..ac354d66206 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/admin/monitoring/VespaMetricSet.java
@@ -142,26 +142,26 @@ public class VespaMetricSet {
addMetric(metrics, ContainerMetrics.HANDLED_REQUESTS.count());
addMetric(metrics, ContainerMetrics.HANDLED_LATENCY, EnumSet.of(sum, count, max));
- addMetric(metrics, ContainerMetrics.SERVER_NUM_OPEN_CONNECTIONS, EnumSet.of(max, last, average));
- addMetric(metrics, ContainerMetrics.SERVER_NUM_CONNECTIONS, EnumSet.of(max, last, average));
+ addMetric(metrics, ContainerMetrics.SERVER_NUM_OPEN_CONNECTIONS, EnumSet.of(max, last, average)); // TODO: Vespa 9: Remove last
+ addMetric(metrics, ContainerMetrics.SERVER_NUM_CONNECTIONS, EnumSet.of(max, last, average)); // TODO: Vespa 9: Remove last
addMetric(metrics, ContainerMetrics.SERVER_BYTES_RECEIVED, EnumSet.of(sum, count));
addMetric(metrics, ContainerMetrics.SERVER_BYTES_SENT, EnumSet.of(sum, count));
- addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_UNHANDLED_EXCEPTIONS, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_WORK_QUEUE_CAPACITY, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_WORK_QUEUE_SIZE, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_REJECTED_TASKS, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_SIZE, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_MAX_ALLOWED_SIZE, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_ACTIVE_THREADS, EnumSet.of(sum, count, last, min, max));
-
- addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_MAX_THREADS, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_MIN_THREADS, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_RESERVED_THREADS, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_BUSY_THREADS, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_TOTAL_THREADS, EnumSet.of(sum, count, last, min, max));
- addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_QUEUE_SIZE, EnumSet.of(sum, count, last, min, max));
+ addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_UNHANDLED_EXCEPTIONS, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove last, min, max
+ addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_WORK_QUEUE_CAPACITY, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove sum, count, last, min
+ addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_WORK_QUEUE_SIZE, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove last
+ addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_REJECTED_TASKS, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove last, min, max
+ addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_SIZE, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove sum, count, last, min
+ addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_MAX_ALLOWED_SIZE, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove sum, count, last, min
+ addMetric(metrics, ContainerMetrics.JDISC_THREAD_POOL_ACTIVE_THREADS, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove last
+
+ addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_MAX_THREADS, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove.
+ addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_MIN_THREADS, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove.
+ addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_RESERVED_THREADS, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove.
+ addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_BUSY_THREADS, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove last, min
+ addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_TOTAL_THREADS, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove sum, count, last, min
+ addMetric(metrics, ContainerMetrics.JETTY_THREADPOOL_QUEUE_SIZE, EnumSet.of(sum, count, last, min, max)); // TODO: Vespa 9: Remove sum, count, last, min
addMetric(metrics, ContainerMetrics.HTTPAPI_LATENCY, EnumSet.of(max, sum, count));
addMetric(metrics, ContainerMetrics.HTTPAPI_PENDING, EnumSet.of(max, sum, count));
@@ -192,8 +192,8 @@ public class VespaMetricSet {
addMetric(metrics, ContainerMetrics.JDISC_MEMORY_MAPPINGS.max());
addMetric(metrics, ContainerMetrics.JDISC_OPEN_FILE_DESCRIPTORS.max());
- addMetric(metrics, ContainerMetrics.JDISC_GC_COUNT, EnumSet.of(average, max, last));
- addMetric(metrics, ContainerMetrics.JDISC_GC_MS, EnumSet.of(average, max, last));
+ addMetric(metrics, ContainerMetrics.JDISC_GC_COUNT, EnumSet.of(average, max, last)); // TODO: Vespa 9: Remove last
+ addMetric(metrics, ContainerMetrics.JDISC_GC_MS, EnumSet.of(average, max, last)); // TODO: Vespa 9: Remove last
addMetric(metrics, ContainerMetrics.JDISC_DEACTIVATED_CONTAINERS.last());
addMetric(metrics, ContainerMetrics.JDISC_DEACTIVATED_CONTAINERS_WITH_RETAINED_REFS.last());
@@ -263,19 +263,19 @@ public class VespaMetricSet {
addMetric(metrics, ClusterControllerMetrics.STOPPING_COUNT.last());
addMetric(metrics, ClusterControllerMetrics.UP_COUNT.last());
addMetric(metrics, ClusterControllerMetrics.CLUSTER_STATE_CHANGE_COUNT.baseName());
- addMetric(metrics, ClusterControllerMetrics.BUSY_TICK_TIME_MS, EnumSet.of(last, max, sum, count));
- addMetric(metrics, ClusterControllerMetrics.IDLE_TICK_TIME_MS, EnumSet.of(last, max, sum, count));
+ addMetric(metrics, ClusterControllerMetrics.BUSY_TICK_TIME_MS, EnumSet.of(last, max, sum, count)); // TODO: Vespa 9: Remove last
+ addMetric(metrics, ClusterControllerMetrics.IDLE_TICK_TIME_MS, EnumSet.of(last, max, sum, count)); // TODO: Vespa 9: Remove last
- addMetric(metrics, ClusterControllerMetrics.WORK_MS, EnumSet.of(last, sum, count));
+ addMetric(metrics, ClusterControllerMetrics.WORK_MS, EnumSet.of(last, sum, count)); // TODO: Vespa 9: Remove last
addMetric(metrics, ClusterControllerMetrics.IS_MASTER.last());
addMetric(metrics, ClusterControllerMetrics.REMOTE_TASK_QUEUE_SIZE.last());
// TODO(hakonhall): Update this name once persistent "count" metrics has been implemented.
// DO NOT RELY ON THIS METRIC YET.
addMetric(metrics, ClusterControllerMetrics.NODE_EVENT_COUNT.baseName());
- addMetric(metrics, ClusterControllerMetrics.RESOURCE_USAGE_NODES_ABOVE_LIMIT, EnumSet.of(last, max));
- addMetric(metrics, ClusterControllerMetrics.RESOURCE_USAGE_MAX_MEMORY_UTILIZATION, EnumSet.of(last, max));
- addMetric(metrics, ClusterControllerMetrics.RESOURCE_USAGE_MAX_DISK_UTILIZATION, EnumSet.of(last, max));
+ addMetric(metrics, ClusterControllerMetrics.RESOURCE_USAGE_NODES_ABOVE_LIMIT, EnumSet.of(last, max)); // TODO: Vespa 9: Remove last
+ addMetric(metrics, ClusterControllerMetrics.RESOURCE_USAGE_MAX_MEMORY_UTILIZATION, EnumSet.of(last, max)); // TODO: Vespa 9: Remove last
+ addMetric(metrics, ClusterControllerMetrics.RESOURCE_USAGE_MAX_DISK_UTILIZATION, EnumSet.of(last, max)); // TODO: Vespa 9: Remove last
addMetric(metrics, ClusterControllerMetrics.RESOURCE_USAGE_MEMORY_LIMIT.last());
addMetric(metrics, ClusterControllerMetrics.RESOURCE_USAGE_DISK_LIMIT.last());
addMetric(metrics, ClusterControllerMetrics.REINDEXING_PROGRESS.last());
@@ -311,7 +311,7 @@ public class VespaMetricSet {
addMetric(metrics, ContainerMetrics.DOCUMENTS_COVERED.count());
addMetric(metrics, ContainerMetrics.DOCUMENTS_TOTAL.count());
addMetric(metrics, ContainerMetrics.DOCUMENTS_TARGET_TOTAL.count());
- addMetric(metrics, ContainerMetrics.JDISC_RENDER_LATENCY, EnumSet.of(min, max, count, sum, last, average));
+ addMetric(metrics, ContainerMetrics.JDISC_RENDER_LATENCY, EnumSet.of(min, max, count, sum, last, average)); // TODO: Vespa 9: Remove last, average
addMetric(metrics, ContainerMetrics.QUERY_ITEM_COUNT, EnumSet.of(max, sum, count));
addMetric(metrics, ContainerMetrics.TOTAL_HITS_PER_QUERY, EnumSet.of(sum, count, max, ninety_five_percentile, ninety_nine_percentile));
addMetric(metrics, ContainerMetrics.EMPTY_RESULTS.rate());
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/container/search/QueryProfiles.java b/config-model/src/main/java/com/yahoo/vespa/model/container/search/QueryProfiles.java
index e2bc9897c85..5184d4ef07a 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/container/search/QueryProfiles.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/container/search/QueryProfiles.java
@@ -112,7 +112,7 @@ public class QueryProfiles implements Serializable, QueryProfilesConfig.Producer
private void addFieldChildren(QueryProfilesConfig.Queryprofile.Builder qpB, QueryProfile profile, String namePrefix) {
List<Map.Entry<String, Object>> content = new ArrayList<>(profile.declaredContent().entrySet());
- Collections.sort(content, new MapEntryKeyComparator());
+ content.sort(new MapEntryKeyComparator());
if (profile.getValue() != null) { // Add "prefix with dot removed"=value:
QueryProfilesConfig.Queryprofile.Property.Builder propB = new QueryProfilesConfig.Queryprofile.Property.Builder();
String fullName = namePrefix.substring(0, namePrefix.length() - 1);
@@ -132,7 +132,7 @@ public class QueryProfiles implements Serializable, QueryProfilesConfig.Producer
QueryProfile profile,
String namePrefix) {
List<Map.Entry<String, Object>> content = new ArrayList<>(profile.declaredContent().entrySet());
- Collections.sort(content,new MapEntryKeyComparator());
+ content.sort(new MapEntryKeyComparator());
if (profile.getValue() != null) { // Add "prefix with dot removed"=value:
QueryProfilesConfig.Queryprofile.Queryprofilevariant.Property.Builder propB = new QueryProfilesConfig.Queryprofile.Queryprofilevariant.Property.Builder();
String fullName = namePrefix.substring(0, namePrefix.length() - 1);
@@ -152,8 +152,7 @@ public class QueryProfiles implements Serializable, QueryProfilesConfig.Producer
private void addField(QueryProfilesConfig.Queryprofile.Builder qpB,
QueryProfile profile, Entry<String, Object> field, String namePrefix) {
String fullName=namePrefix + field.getKey();
- if (field.getValue() instanceof QueryProfile) {
- QueryProfile subProfile=(QueryProfile)field.getValue();
+ if (field.getValue() instanceof QueryProfile subProfile) {
if ( ! subProfile.isExplicit()) { // Implicitly defined profile - add content
addFieldChildren(qpB, subProfile,fullName + ".");
}
@@ -172,8 +171,7 @@ public class QueryProfiles implements Serializable, QueryProfilesConfig.Producer
private void addVariantField(QueryProfilesConfig.Queryprofile.Queryprofilevariant.Builder qpB,
Entry<String, Object> field, Boolean overridable, String namePrefix) {
String fullName = namePrefix + field.getKey();
- if (field.getValue() instanceof QueryProfile) {
- QueryProfile subProfile = (QueryProfile)field.getValue();
+ if (field.getValue() instanceof QueryProfile subProfile) {
if ( ! subProfile.isExplicit()) { // Implicitly defined profile - add content
addVariantFieldChildren(qpB, subProfile,fullName + ".");
}
@@ -203,7 +201,7 @@ public class QueryProfiles implements Serializable, QueryProfilesConfig.Producer
varB.inherit(inherited.getId().stringValue());
List<Map.Entry<String,Object>> content = new ArrayList<>(variant.getValues().entrySet());
- Collections.sort(content, new MapEntryKeyComparator());
+ content.sort(new MapEntryKeyComparator());
for (Map.Entry<String, Object> entry : content) {
addVariantField(varB, entry, variant.getOverriable().get(entry.getKey()), "");
}
diff --git a/config-model/src/main/java/com/yahoo/vespa/model/search/IndexedSearchCluster.java b/config-model/src/main/java/com/yahoo/vespa/model/search/IndexedSearchCluster.java
index 4ed8c5ab2e8..670460a9f9f 100644
--- a/config-model/src/main/java/com/yahoo/vespa/model/search/IndexedSearchCluster.java
+++ b/config-model/src/main/java/com/yahoo/vespa/model/search/IndexedSearchCluster.java
@@ -64,6 +64,7 @@ public class IndexedSearchCluster extends SearchCluster
private final List<SearchNode> searchNodes = new ArrayList<>();
private final DispatchTuning.DispatchPolicy defaultDispatchPolicy;
private final double dispatchWarmup;
+ private final String summaryDecodePolicy;
/**
* Returns the document selector that is able to resolve what documents are to be routed to this search cluster.
* This string uses the document selector language as defined in the "document" module.
@@ -80,6 +81,7 @@ public class IndexedSearchCluster extends SearchCluster
rootDispatch = new DispatchGroup(this);
defaultDispatchPolicy = DispatchTuning.Builder.toDispatchPolicy(featureFlags.queryDispatchPolicy());
dispatchWarmup = featureFlags.queryDispatchWarmup();
+ summaryDecodePolicy = featureFlags.summaryDecodePolicy();
}
@Override
@@ -337,6 +339,15 @@ public class IndexedSearchCluster extends SearchCluster
builder.maxWaitAfterCoverageFactor(searchCoverage.getMaxWaitAfterCoverageFactor());
}
builder.warmuptime(dispatchWarmup);
+ builder.summaryDecodePolicy(toSummaryDecoding(summaryDecodePolicy));
+ }
+
+ private DispatchConfig.SummaryDecodePolicy.Enum toSummaryDecoding(String summaryDecodeType) {
+ return switch (summaryDecodeType.toLowerCase()) {
+ case "eager" -> DispatchConfig.SummaryDecodePolicy.EAGER;
+ case "ondemand","on-demand" -> DispatchConfig.SummaryDecodePolicy.Enum.ONDEMAND;
+ default -> DispatchConfig.SummaryDecodePolicy.Enum.EAGER;
+ };
}
@Override
diff --git a/config-model/src/main/javacc/SchemaParser.jj b/config-model/src/main/javacc/SchemaParser.jj
index 61e8574bc87..9d6e16b3f67 100644
--- a/config-model/src/main/javacc/SchemaParser.jj
+++ b/config-model/src/main/javacc/SchemaParser.jj
@@ -164,13 +164,13 @@ TOKEN :
| < FIELD: "field" >
| < FIELDS: "fields" >
| < FIELDSET: "fieldset" >
-| < STRUCTFIELD: "struct-field" >
+| < STRUCT_FIELD: "struct-field" >
| < IMPORT: "import" >
| < AS: "as" >
| < INDEXING: "indexing" >
-| < SUMMARYTO: "summary-to" >
-| < DOCUMENTSUMMARY: "document-summary" >
-| < RANKTYPE: "rank-type" >
+| < SUMMARY_TO: "summary-to" >
+| < DOCUMENT_SUMMARY: "document-summary" >
+| < RANK_TYPE: "rank-type" >
| < WEIGHT: "weight" >
| < TYPE: "type" >
| < INDEX: "index" >
@@ -179,26 +179,29 @@ TOKEN :
| < TEXT: "text" >
| < WORD: "word" >
| < GRAM: "gram" >
-| < GRAMSIZE: "gram-size" >
-| < MAXLENGTH: "max-length" >
+| < GRAM_SIZE: "gram-size" >
+| < MAX_LENGTH: "max-length" >
| < PREFIX: "prefix" >
| < SUBSTRING: "substring" >
| < SUFFIX: "suffix" >
| < CONSTANT: "constant">
-| < ONNXMODEL: "onnx-model">
-| < INTRAOPTHREADS: "intraop-threads">
-| < INTEROPTHREADS: "interop-threads">
-| < GPUDEVICE: "gpu-device">
+| < ONNX_MODEL: "onnx-model">
+| < INTRAOP_THREADS: "intraop-threads">
+| < INTEROP_THREADS: "interop-threads">
+| < GPU_DEVICE: "gpu-device">
+| < EXECUTION_MODE: "execution-mode">
+| < PARALLEL: "parallel">
+| < SEQUENTIAL: "sequential">
| < MODEL: "model" >
| < MUTATE: "mutate" >
| < QUERY: "query" >
-| < RANKPROFILE: "rank-profile" >
+| < RANK_PROFILE: "rank-profile" >
| < RAW_AS_BASE64_IN_SUMMARY: "raw-as-base64-in-summary" >
| < SUMMARY: "summary" >
| < FULL: "full" >
| < STATIC: "static" >
| < DYNAMIC: "dynamic" >
-| < MATCHEDELEMENTSONLY: "matched-elements-only" >
+| < MATCHED_ELEMENTS_ONLY: "matched-elements-only" >
| < SSCONTEXTUAL: "contextual" >
| < SSOVERRIDE: "override" >
| < SSTITLE: "title" >
@@ -233,7 +236,7 @@ TOKEN :
| < TRUE: "true" >
| < FALSE: "false" >
| < SYMMETRIC: "symmetric" >
-| < QUERYCOMMAND: "query-command" >
+| < QUERY_COMMAND: "query-command" >
| < ALIAS: "alias" >
| < MATCH: "match" >
| < RANK: "rank" >
@@ -241,24 +244,24 @@ TOKEN :
| < EXACT: "exact" >
| < FILTER: "filter" >
| < NORMAL: "normal" >
-| < EXACTTERMINATOR: "exact-terminator" >
-| < IGNOREDEFAULTRANKFEATURES: "ignore-default-rank-features" >
+| < EXACT_TERMINATOR: "exact-terminator" >
+| < IGNORE_DEFAULT_RANK_FEATURES: "ignore-default-rank-features" >
| < ID: "id" >
| < SOURCE: "source" >
| < TO: "to" >
| < DIRECT: "direct" >
-| < FROMDISK: "from-disk" >
-| < OMITSUMMARYFEATURES: "omit-summary-features" >
+| < FROM_DISK: "from-disk" >
+| < OMIT_SUMMARY_FEATURES: "omit-summary-features" >
| < ALWAYS: "always" >
-| < ONDEMAND: "on-demand" >
+| < ON_DEMAND: "on-demand" >
| < NEVER: "never" >
-| < ENABLEBITVECTORS: "enable-bit-vectors" >
-| < ENABLEONLYBITVECTOR: "enable-only-bit-vector" >
-| < FASTACCESS: "fast-access" >
+| < ENABLE_BIT_VECTORS: "enable-bit-vectors" >
+| < ENABLE_ONLY_BIT_VECTOR: "enable-only-bit-vector" >
+| < FAST_ACCESS: "fast-access" >
| < MUTABLE: "mutable" >
| < PAGED: "paged" >
-| < FASTRANK: "fast-rank" >
-| < FASTSEARCH: "fast-search" >
+| < FAST_RANK: "fast-rank" >
+| < FAST_SEARCH: "fast-search" >
| < TENSOR_TYPE: "tensor" ("<" (~["<",">"])+ ">")? "(" (~["(",")"])* ")" >
| < TENSOR_VALUE_SL: "value" (" ")* ":" (" ")* ("{"<BRACE_SL_LEVEL_1>) ("\n")? >
| < TENSOR_VALUE_ML: "value" (<SEARCHLIB_SKIP>)? "{" (["\n"," "])* ("{"<BRACE_ML_LEVEL_1>) (["\n"," "])* "}" ("\n")? >
@@ -272,30 +275,30 @@ TOKEN :
| < MAP: "map" >
| < REFERENCE: "reference" >
| < QUESTIONMARK: "?" >
-| < CREATEIFNONEXISTENT: "create-if-nonexistent" >
-| < REMOVEIFZERO: "remove-if-zero" >
-| < MATCHPHASE: "match-phase" >
+| < CREATE_IF_NONEXISTENT: "create-if-nonexistent" >
+| < REMOVE_IF_ZERO: "remove-if-zero" >
+| < MATCH_PHASE: "match-phase" >
| < EVALUATION_POINT: "evaluation-point" >
| < PRE_POST_FILTER_TIPPING_POINT: "pre-post-filter-tipping-point" >
| < ORDER: "order" >
-| < MAXFILTERCOVERAGE: "max-filter-coverage" >
-| < MAXHITS: "max-hits" >
-| < FIRSTPHASE: "first-phase" >
-| < SECONDPHASE: "second-phase" >
-| < GLOBALPHASE: "global-phase" >
+| < MAX_FILTER_COVERAGE: "max-filter-coverage" >
+| < MAX_HITS: "max-hits" >
+| < FIRST_PHASE: "first-phase" >
+| < SECOND_PHASE: "second-phase" >
+| < GLOBAL_PHASE: "global-phase" >
| < MACRO: "macro" >
| < INLINE: "inline" >
| < ARITY: "arity" >
-| < LOWERBOUND: "lower-bound" >
-| < UPPERBOUND: "upper-bound" >
-| < DENSEPOSTINGLISTTHRESHOLD: "dense-posting-list-threshold" >
+| < LOWER_BOUND: "lower-bound" >
+| < UPPER_BOUND: "upper-bound" >
+| < DENSE_POSTING_LIST_THRESHOLD: "dense-posting-list-threshold" >
| < ENABLE_BM25: "enable-bm25" >
| < HNSW: "hnsw" >
-| < MAXLINKSPERNODE: "max-links-per-node" >
+| < MAX_LINKS_PER_NODE: "max-links-per-node" >
| < DOUBLE_KEYWORD: "double" >
-| < DISTANCEMETRIC: "distance-metric" >
-| < NEIGHBORSTOEXPLOREATINSERT: "neighbors-to-explore-at-insert" >
-| < MULTITHREADEDINDEXING: "multi-threaded-indexing" >
+| < DISTANCE_METRIC: "distance-metric" >
+| < NEIGHBORS_TO_EXPLORE_AT_INSERT: "neighbors-to-explore-at-insert" >
+| < MULTI_THREADED_INDEXING: "multi-threaded-indexing" >
| < MATCHFEATURES_SL: "match-features" (" ")* ":" (~["}","\n"])* ("\n")? >
| < MATCHFEATURES_ML: "match-features" (<SEARCHLIB_SKIP>)? "{" (~["}"])* "}" >
| < MATCHFEATURES_ML_INHERITS: "match-features inherits " (<IDENTIFIER_WITH_DASH>) (<SEARCHLIB_SKIP>)? "{" (~["}"])* "}" >
@@ -315,16 +318,16 @@ TOKEN :
| < #BRACE_ML_LEVEL_3: <BRACE_ML_CONTENT> "}" >
| < #BRACE_ML_CONTENT: (~["{","}"])* >
| < #SEARCHLIB_SKIP: ([" ","\f","\n","\r","\t"])+ >
-| < RANKPROPERTIES: "rank-properties" >
-| < RERANKCOUNT: "rerank-count" >
-| < NUMTHREADSPERSEARCH: "num-threads-per-search" >
-| < MINHITSPERTHREAD: "min-hits-per-thread" >
-| < NUMSEARCHPARTITIONS: "num-search-partitions" >
-| < TERMWISELIMIT: "termwise-limit" >
-| < POSTFILTERTHRESHOLD: "post-filter-threshold" >
-| < APPROXIMATETHRESHOLD: "approximate-threshold" >
-| < KEEPRANKCOUNT: "keep-rank-count" >
-| < RANKSCOREDROPLIMIT: "rank-score-drop-limit" >
+| < RANK_PROPERTIES: "rank-properties" >
+| < RERANK_COUNT: "rerank-count" >
+| < NUM_THREADS_PER_SEARCH: "num-threads-per-search" >
+| < MIN_HITS_PER_THREAD: "min-hits-per-thread" >
+| < NUM_SEARCH_PARTITIONS: "num-search-partitions" >
+| < TERMWISE_LIMIT: "termwise-limit" >
+| < POST_FILTER_THRESHOLD: "post-filter-threshold" >
+| < APPROXIMATE_THRESHOLD: "approximate-threshold" >
+| < KEEP_RANK_COUNT: "keep-rank-count" >
+| < RANK_SCORE_DROP_LIMIT: "rank-score-drop-limit" >
| < CONSTANTS: "constants" >
| < FILE: "file" >
| < URI: "uri" >
@@ -608,7 +611,7 @@ void fieldSet(ParsedSchema schema) :
((
( <FIELDS><COLON> name = identifier() { fieldSet.addField(name); }
( <COMMA> name = identifier() { fieldSet.addField(name); } )* )
- | ( <QUERYCOMMAND> <COLON> (queryCommand = identifierWithDash() | queryCommand = quotedString())) { fieldSet.addQueryCommand(queryCommand); }
+ | ( <QUERY_COMMAND> <COLON> (queryCommand = identifierWithDash() | queryCommand = quotedString())) { fieldSet.addQueryCommand(queryCommand); }
| matchSettings(fieldSet.matchSettings())
) (<NL>)* )+
<RBRACE>
@@ -825,7 +828,7 @@ void structField(ParsedField field) :
ParsedField structField;
}
{
- <STRUCTFIELD> name = identifier() {
+ <STRUCT_FIELD> name = identifier() {
if (name != null && Schema.isReservedName(name.toLowerCase())) {
throw new IllegalArgumentException("Reserved name '" + name + "' can not be used as a field name.");
}
@@ -917,7 +920,7 @@ void summaryTo(ParsedField field) :
ParsedSummaryField psf;
}
{
- <SUMMARYTO> [name = identifier()] <COLON> destination = identifier()
+ <SUMMARY_TO> [name = identifier()] <COLON> destination = identifier()
{
psf = field.summaryFieldFor(name);
psf.addDestination(destination);
@@ -956,8 +959,8 @@ void weightedset(ParsedType fieldType) : { }
*/
void weightedsetBody(ParsedType type) : { }
{
- ( <CREATEIFNONEXISTENT> { type.setCreateIfNonExistent(true); }
- | <REMOVEIFZERO> { type.setRemoveIfZero(true); } )
+ ( <CREATE_IF_NONEXISTENT> { type.setCreateIfNonExistent(true); }
+ | <REMOVE_IF_ZERO> { type.setRemoveIfZero(true); } )
}
/**
@@ -971,7 +974,7 @@ void rankType(ParsedField field) :
String indexName = "";
}
{
- <RANKTYPE> [indexName = identifier()] <COLON> typeName = identifier()
+ <RANK_TYPE> [indexName = identifier()] <COLON> typeName = identifier()
{
field.addRankType(indexName, typeName);
}
@@ -1049,18 +1052,18 @@ void attributeSetting(ParsedAttribute attribute) :
}
{
(
- <FASTRANK> { attribute.setFastRank(true); }
- | <FASTSEARCH> { attribute.setFastSearch(true); }
- | <FASTACCESS> { attribute.setFastAccess(true); }
+ <FAST_RANK> { attribute.setFastRank(true); }
+ | <FAST_SEARCH> { attribute.setFastSearch(true); }
+ | <FAST_ACCESS> { attribute.setFastAccess(true); }
| <MUTABLE> { attribute.setMutable(true); }
| <PAGED> { attribute.setPaged(true); }
- | <ENABLEBITVECTORS> { deployLogger.logApplicationPackage(Level.WARNING, "'enable-bit-vectors' is deprecated and void -> remove it. Will be removed in vespa-9"); }
- | <ENABLEONLYBITVECTOR> { attribute.setEnableOnlyBitVector(true); }
+ | <ENABLE_BIT_VECTORS> { deployLogger.logApplicationPackage(Level.WARNING, "'enable-bit-vectors' is deprecated and void -> remove it. Will be removed in vespa-9"); }
+ | <ENABLE_ONLY_BIT_VECTOR> { attribute.setEnableOnlyBitVector(true); }
| attributeSorting(attribute)
| <ALIAS> { String alias; String aliasedName=attribute.name(); } [aliasedName = identifier()] <COLON> alias = identifierWithDash() {
attribute.addAlias(aliasedName, alias);
}
- | <DISTANCEMETRIC> <COLON> str = identifierWithDash() { attribute.setDistanceMetric(str); }
+ | <DISTANCE_METRIC> <COLON> str = identifierWithDash() { attribute.setDistanceMetric(str); }
)
}
@@ -1116,7 +1119,7 @@ void summaryInFieldShort(ParsedField field) :
psf = field.summaryFieldFor(name);
}
<COLON> ( <DYNAMIC> { psf.setDynamic(); }
- | <MATCHEDELEMENTSONLY> { psf.setMatchedElementsOnly(); }
+ | <MATCHED_ELEMENTS_ONLY> { psf.setMatchedElementsOnly(); }
| (<FULL> | <STATIC>) { psf.setFull(); }
)
}
@@ -1161,7 +1164,7 @@ void summaryItem(ParsedSummaryField field) : { }
void summaryTransform(ParsedSummaryField field) : { }
{
( <DYNAMIC> { field.setDynamic(); }
- | <MATCHEDELEMENTSONLY> { field.setMatchedElementsOnly(); }
+ | <MATCHED_ELEMENTS_ONLY> { field.setMatchedElementsOnly(); }
| (<FULL> | <STATIC>) { field.setFull(); }
)
}
@@ -1295,7 +1298,7 @@ void queryCommand(ParsedField field) :
String command;
}
{
- <QUERYCOMMAND> <COLON> ( command = identifierWithDash() | command = quotedString() )
+ <QUERY_COMMAND> <COLON> ( command = identifierWithDash() | command = quotedString() )
{
field.addQueryCommand(command);
}
@@ -1349,7 +1352,7 @@ void exactTerminator(ParsedMatchSettings matchInfo) :
String terminator;
}
{
- <EXACTTERMINATOR> <COLON> terminator = quotedString()
+ <EXACT_TERMINATOR> <COLON> terminator = quotedString()
{
matchInfo.setExactTerminator(terminator);
}
@@ -1360,7 +1363,7 @@ void gramSize(ParsedMatchSettings matchInfo) :
int gramSize;
}
{
- <GRAMSIZE> <COLON> gramSize = integer()
+ <GRAM_SIZE> <COLON> gramSize = integer()
{
matchInfo.setGramSize(gramSize);
}
@@ -1371,7 +1374,7 @@ void matchSize(ParsedMatchSettings matchInfo) :
int matchSize;
}
{
- <MAXLENGTH> <COLON> matchSize = integer() {
+ <MAX_LENGTH> <COLON> matchSize = integer() {
matchInfo.setMaxLength(matchSize);
}
}
@@ -1426,13 +1429,13 @@ void documentSummary(ParsedSchema schema) :
ParsedDocumentSummary summary;
}
{
- ( <DOCUMENTSUMMARY>
+ ( <DOCUMENT_SUMMARY>
name = identifierWithDash() { summary = new ParsedDocumentSummary(name); }
[inheritsDocumentSummary(summary)]
lbrace()
(
- <FROMDISK> { summary.setFromDisk(true); } |
- <OMITSUMMARYFEATURES> { summary.setOmitSummaryFeatures(true); } |
+ <FROM_DISK> { summary.setFromDisk(true); } |
+ <OMIT_SUMMARY_FEATURES> { summary.setOmitSummaryFeatures(true); } |
documentSummaryItem(summary) |
<NL>
)*
@@ -1529,9 +1532,9 @@ void indexBody(ParsedIndex index) :
| <ALIAS> <COLON> str = identifierWithDash() { index.addAlias(str); }
| <STEMMING> <COLON> str = identifierWithDash() { index.setStemming(Stemming.get(str)); }
| <ARITY> <COLON> arity = integer() { index.setArity(arity); }
- | <LOWERBOUND> <COLON> num = longValue() { index.setLowerBound(num); }
- | <UPPERBOUND> <COLON> num = longValue() { index.setUpperBound(num); }
- | <DENSEPOSTINGLISTTHRESHOLD> <COLON> threshold = floatValue() { index.setDensePostingListThreshold(threshold); }
+ | <LOWER_BOUND> <COLON> num = longValue() { index.setLowerBound(num); }
+ | <UPPER_BOUND> <COLON> num = longValue() { index.setUpperBound(num); }
+ | <DENSE_POSTING_LIST_THRESHOLD> <COLON> threshold = floatValue() { index.setDensePostingListThreshold(threshold); }
| <ENABLE_BM25> { index.setEnableBm25(true); }
| hnswIndex(index) { }
)
@@ -1556,9 +1559,9 @@ void hnswIndexBody(HnswIndexParams.Builder params) :
boolean bool;
}
{
- ( <MAXLINKSPERNODE> <COLON> num = integer() { params.setMaxLinksPerNode(num); }
- | <NEIGHBORSTOEXPLOREATINSERT> <COLON> num = integer() { params.setNeighborsToExploreAtInsert(num); }
- | <MULTITHREADEDINDEXING> <COLON> bool = bool() { params.setMultiThreadedIndexing(bool); } )
+ ( <MAX_LINKS_PER_NODE> <COLON> num = integer() { params.setMaxLinksPerNode(num); }
+ | <NEIGHBORS_TO_EXPLORE_AT_INSERT> <COLON> num = integer() { params.setNeighborsToExploreAtInsert(num); }
+ | <MULTI_THREADED_INDEXING> <COLON> bool = bool() { params.setMultiThreadedIndexing(bool); } )
}
void onnxModelInSchema(ParsedSchema schema) :
@@ -1584,7 +1587,7 @@ OnnxModel onnxModel() :
OnnxModel onnxModel;
}
{
- ( <ONNXMODEL> name = identifier() { onnxModel = new OnnxModel(name); }
+ ( <ONNX_MODEL> name = identifier() { onnxModel = new OnnxModel(name); }
lbrace() (onnxModelItem(onnxModel) (<NL>)*)+ <RBRACE> )
{ return onnxModel; }
}
@@ -1603,9 +1606,11 @@ void onnxModelItem(OnnxModel onnxModel) :
(
(path = fileItem()) { onnxModel.setFileName(path); } |
(path = uriItem()) { onnxModel.setUri(path); } |
- <GPUDEVICE> <COLON> num = integer() { onnxModel.setGpuDevice(num, false); } |
- <INTRAOPTHREADS> <COLON> num = integer() { onnxModel.setStatelessIntraOpThreads(num); } |
- <INTEROPTHREADS> <COLON> num = integer() { onnxModel.setStatelessInterOpThreads(num); } |
+ <GPU_DEVICE> <COLON> num = integer() { onnxModel.setGpuDevice(num, false); } |
+ <INTRAOP_THREADS> <COLON> num = integer() { onnxModel.setStatelessIntraOpThreads(num); } |
+ <INTEROP_THREADS> <COLON> num = integer() { onnxModel.setStatelessInterOpThreads(num); } |
+ <EXECUTION_MODE> <COLON> ( <PARALLEL> { onnxModel.setStatelessExecutionMode("parallel"); }
+ | <SEQUENTIAL> { onnxModel.setStatelessExecutionMode("sequential"); } ) |
(<ONNX_INPUT_SL>) {
String name = token.image.substring(5, token.image.lastIndexOf(":")).trim();
if (name.startsWith("\"")) { name = name.substring(1, name.length() - 1); }
@@ -1680,7 +1685,7 @@ void rankProfile(ParsedSchema schema) :
ParsedRankProfile profile;
}
{
- ( <MODEL> | <RANKPROFILE> ) name = identifierWithDash()
+ ( <MODEL> | <RANK_PROFILE> ) name = identifierWithDash()
{ profile = new ParsedRankProfile(name); }
[inheritsRankProfile(profile)]
lbrace() (rankProfileItem(schema, profile) (<NL>)*)* <RBRACE>
@@ -1819,7 +1824,7 @@ void matchPhase(ParsedRankProfile profile) :
MatchPhaseSettings settings = new MatchPhaseSettings();
}
{
- <MATCHPHASE> lbrace() (matchPhaseItem(settings) (<NL>)*)* <RBRACE>
+ <MATCH_PHASE> lbrace() (matchPhaseItem(settings) (<NL>)*)* <RBRACE>
{
settings.checkValid();
profile.setMatchPhaseSettings(settings);
@@ -1838,8 +1843,8 @@ void matchPhaseItem(MatchPhaseSettings settings) :
| diversity(settings)
| <ORDER> <COLON> ( <ASCENDING> { settings.setAscending(true); }
| <DESCENDING> { settings.setAscending(false); } )
- | <MAXHITS> <COLON> num = integer() { settings.setMaxHits(num); }
- | <MAXFILTERCOVERAGE> <COLON> coverage = floatValue() { settings.setMaxFilterCoverage(coverage); }
+ | <MAX_HITS> <COLON> num = integer() { settings.setMaxHits(num); }
+ | <MAX_FILTER_COVERAGE> <COLON> coverage = floatValue() { settings.setMaxFilterCoverage(coverage); }
| <EVALUATION_POINT> <COLON> multiplier = floatValue() { settings.setEvaluationPoint(multiplier); }
| <PRE_POST_FILTER_TIPPING_POINT> <COLON> multiplier = floatValue() { settings.setPrePostFilterTippingPoint(multiplier); }
)
@@ -1890,7 +1895,7 @@ void firstPhase(ParsedRankProfile profile) :
String exp;
}
{
- <FIRSTPHASE> lbrace() (firstPhaseItem(profile) (<NL>)*)* <RBRACE>
+ <FIRST_PHASE> lbrace() (firstPhaseItem(profile) (<NL>)*)* <RBRACE>
}
void firstPhaseItem(ParsedRankProfile profile) :
@@ -1901,8 +1906,8 @@ void firstPhaseItem(ParsedRankProfile profile) :
}
{
( expression = expression() { profile.setFirstPhaseRanking(expression); }
- | (<KEEPRANKCOUNT> <COLON> keepRankCount = integer()) { profile.setKeepRankCount(keepRankCount); }
- | (<RANKSCOREDROPLIMIT> <COLON> dropLimit = floatValue()) { profile.setRankScoreDropLimit(dropLimit); }
+ | (<KEEP_RANK_COUNT> <COLON> keepRankCount = integer()) { profile.setKeepRankCount(keepRankCount); }
+ | (<RANK_SCORE_DROP_LIMIT> <COLON> dropLimit = floatValue()) { profile.setRankScoreDropLimit(dropLimit); }
)
}
@@ -1913,7 +1918,7 @@ void firstPhaseItem(ParsedRankProfile profile) :
*/
void secondPhase(ParsedRankProfile profile) : { }
{
- <SECONDPHASE> lbrace() (secondPhaseItem(profile) (<NL>)*)* <RBRACE>
+ <SECOND_PHASE> lbrace() (secondPhaseItem(profile) (<NL>)*)* <RBRACE>
}
/**
@@ -1928,7 +1933,7 @@ void secondPhaseItem(ParsedRankProfile profile) :
}
{
( expression = expression() { profile.setSecondPhaseRanking(expression); }
- | (<RERANKCOUNT> <COLON> rerankCount = integer()) { profile.setRerankCount(rerankCount); }
+ | (<RERANK_COUNT> <COLON> rerankCount = integer()) { profile.setRerankCount(rerankCount); }
)
}
@@ -1939,7 +1944,7 @@ void secondPhaseItem(ParsedRankProfile profile) :
*/
void globalPhase(ParsedRankProfile profile) : { }
{
- <GLOBALPHASE> lbrace() (globalPhaseItem(profile) (<NL>)*)* <RBRACE>
+ <GLOBAL_PHASE> lbrace() (globalPhaseItem(profile) (<NL>)*)* <RBRACE>
}
/**
@@ -1954,7 +1959,7 @@ void globalPhaseItem(ParsedRankProfile profile) :
}
{
( expression = expression() { profile.setGlobalPhaseExpression(expression); }
- | (<RERANKCOUNT> <COLON> rerankCount = integer()) { profile.setGlobalPhaseRerankCount(rerankCount); }
+ | (<RERANK_COUNT> <COLON> rerankCount = integer()) { profile.setGlobalPhaseRerankCount(rerankCount); }
)
}
@@ -2092,7 +2097,7 @@ void rankFeatures(ParsedRankProfile profile) :
*/
void ignoreRankFeatures(ParsedRankProfile profile) : { }
{
- <IGNOREDEFAULTRANKFEATURES> { profile.setIgnoreDefaultRankFeatures(true); }
+ <IGNORE_DEFAULT_RANK_FEATURES> { profile.setIgnoreDefaultRankFeatures(true); }
}
/**
@@ -2105,7 +2110,7 @@ void numThreadsPerSearch(ParsedRankProfile profile) :
int num;
}
{
- (<NUMTHREADSPERSEARCH> <COLON> num = integer()) { profile.setNumThreadsPerSearch(num); }
+ (<NUM_THREADS_PER_SEARCH> <COLON> num = integer()) { profile.setNumThreadsPerSearch(num); }
}
/**
@@ -2118,7 +2123,7 @@ void minHitsPerThread(ParsedRankProfile profile) :
int num;
}
{
- (<MINHITSPERTHREAD> <COLON> num = integer()) { profile.setMinHitsPerThread(num); }
+ (<MIN_HITS_PER_THREAD> <COLON> num = integer()) { profile.setMinHitsPerThread(num); }
}
/**
@@ -2131,7 +2136,7 @@ void numSearchPartitions(ParsedRankProfile profile) :
int num;
}
{
- (<NUMSEARCHPARTITIONS> <COLON> num = integer()) { profile.setNumSearchPartitions(num); }
+ (<NUM_SEARCH_PARTITIONS> <COLON> num = integer()) { profile.setNumSearchPartitions(num); }
}
/**
@@ -2144,7 +2149,7 @@ void termwiseLimit(ParsedRankProfile profile) :
double num;
}
{
- (<TERMWISELIMIT> <COLON> num = floatValue()) { profile.setTermwiseLimit(num); }
+ (<TERMWISE_LIMIT> <COLON> num = floatValue()) { profile.setTermwiseLimit(num); }
}
/**
@@ -2157,7 +2162,7 @@ void postFilterThreshold(ParsedRankProfile profile) :
double threshold;
}
{
- (<POSTFILTERTHRESHOLD> <COLON> threshold = floatValue()) { profile.setPostFilterThreshold(threshold); }
+ (<POST_FILTER_THRESHOLD> <COLON> threshold = floatValue()) { profile.setPostFilterThreshold(threshold); }
}
/**
@@ -2170,7 +2175,7 @@ void approximateThreshold(ParsedRankProfile profile) :
double threshold;
}
{
- (<APPROXIMATETHRESHOLD> <COLON> threshold = floatValue()) { profile.setApproximateThreshold(threshold); }
+ (<APPROXIMATE_THRESHOLD> <COLON> threshold = floatValue()) { profile.setApproximateThreshold(threshold); }
}
/**
@@ -2182,7 +2187,7 @@ void approximateThreshold(ParsedRankProfile profile) :
*/
void rankProperties(ParsedRankProfile profile) : { }
{
- <RANKPROPERTIES> lbrace() (LOOKAHEAD(rankPropertyItem() <COLON> rankPropertyItem() <NL>)
+ <RANK_PROPERTIES> lbrace() (LOOKAHEAD(rankPropertyItem() <COLON> rankPropertyItem() <NL>)
rankProperty(profile) (<NL>)+)* [rankProperty(profile)] <RBRACE>
}
@@ -2245,7 +2250,7 @@ void fieldRankType(ParsedRankProfile profile) :
String type;
}
{
- <RANKTYPE> name = identifier() <COLON> type = identifier()
+ <RANK_TYPE> name = identifier() <COLON> type = identifier()
{ profile.addFieldRankType(name, type); }
}
@@ -2566,7 +2571,68 @@ String identifierWithDash() :
{
( identifier = identifier() { return identifier; } )
|
- ( <IDENTIFIER_WITH_DASH> { return token.image; } )
+ ( <IDENTIFIER_WITH_DASH>
+ | <APPROXIMATE_THRESHOLD>
+ | <CREATE_IF_NONEXISTENT>
+ | <CUTOFF_FACTOR>
+ | <CUTOFF_STRATEGY>
+ | <DENSE_POSTING_LIST_THRESHOLD>
+ | <DISTANCE_METRIC>
+ | <DOCUMENT_SUMMARY>
+ | <ENABLE_BIT_VECTORS>
+ | <ENABLE_BM25>
+ | <ENABLE_ONLY_BIT_VECTOR>
+ | <EVALUATION_POINT>
+ | <EXACT_TERMINATOR>
+ | <EXECUTION_MODE>
+ | <FAST_ACCESS>
+ | <FAST_RANK>
+ | <FAST_SEARCH>
+ | <FIRST_PHASE>
+ | <FROM_DISK>
+ | <GLOBAL_PHASE>
+ | <GPU_DEVICE>
+ | <GRAM_SIZE>
+ | <IGNORE_DEFAULT_RANK_FEATURES>
+ | <INTEROP_THREADS>
+ | <INTRAOP_THREADS>
+ | <KEEP_RANK_COUNT>
+ | <LOWER_BOUND>
+ | <MATCHED_ELEMENTS_ONLY>
+ | <MATCH_PHASE>
+ | <MAX_FILTER_COVERAGE>
+ | <MAX_HITS>
+ | <MAX_LENGTH>
+ | <MAX_LINKS_PER_NODE>
+ | <MIN_GROUPS>
+ | <MIN_HITS_PER_THREAD>
+ | <MULTI_THREADED_INDEXING>
+ | <NEIGHBORS_TO_EXPLORE_AT_INSERT>
+ | <NUM_SEARCH_PARTITIONS>
+ | <NUM_THREADS_PER_SEARCH>
+ | <OMIT_SUMMARY_FEATURES>
+ | <ON_DEMAND>
+ | <ON_FIRST_PHASE>
+ | <ON_MATCH>
+ | <ONNX_MODEL>
+ | <ON_SECOND_PHASE>
+ | <ON_SUMMARY>
+ | <POST_FILTER_THRESHOLD>
+ | <PRE_POST_FILTER_TIPPING_POINT>
+ | <QUERY_COMMAND>
+ | <RANK_PROFILE>
+ | <RANK_PROPERTIES>
+ | <RANK_SCORE_DROP_LIMIT>
+ | <RANK_TYPE>
+ | <RAW_AS_BASE64_IN_SUMMARY>
+ | <REMOVE_IF_ZERO>
+ | <RERANK_COUNT>
+ | <SECOND_PHASE>
+ | <STRUCT_FIELD>
+ | <SUMMARY_TO>
+ | <TERMWISE_LIMIT>
+ | <UPPER_BOUND>
+ ) { return token.image; }
}
/**
@@ -2592,38 +2658,28 @@ String identifier() : { }
| <CONSTANT>
| <CONSTANTS>
| <CONTEXT>
- | <GLOBALPHASE>
- | <CREATEIFNONEXISTENT>
- | <DENSEPOSTINGLISTTHRESHOLD>
| <DESCENDING>
| <DICTIONARY>
| <DIRECT>
+ | <DIVERSITY>
| <DOCUMENT>
- | <DOCUMENTSUMMARY>
| <DOUBLE_KEYWORD>
| <DYNAMIC>
- | <ENABLEBITVECTORS>
- | <ENABLEONLYBITVECTOR>
| <EXACT>
- | <EXACTTERMINATOR>
| <FALSE>
- | <FASTACCESS>
- | <FASTRANK>
- | <FASTSEARCH>
| <FIELD>
| <FIELDS>
| <FIELDSET>
| <FILE>
| <FILTER>
- | <FIRSTPHASE>
| <FULL>
| <FUNCTION>
| <GRAM>
| <HASH>
+ | <HNSW>
| <ID>
| <IDENTICAL>
| <IDENTIFIER>
- | <IGNOREDEFAULTRANKFEATURES>
| <IMPORT>
| <INDEX>
| <INDEXING>
@@ -2631,52 +2687,42 @@ String identifier() : { }
| <INLINE>
| <INPUTS>
| <INTEGER>
- | <KEEPRANKCOUNT>
| <LITERAL>
| <LOCALE>
| <LONG>
| <LOOSE>
- | <LOWERBOUND>
| <LOWERCASE>
| <MACRO>
| <MAP>
| <MATCH>
- | <MATCHPHASE>
- | <MAXFILTERCOVERAGE>
- | <MAXHITS>
+ | <MODEL>
| <MTOKEN>
| <MUTABLE>
+ | <MUTATE>
| <NEVER>
| <NONE>
| <NORMAL>
| <NORMALIZING>
| <OFF>
| <ON>
- | <ONDEMAND>
+ | <OPERATION>
| <ORDER>
+ | <PAGED>
+ | <PARALLEL>
| <PREFIX>
| <PRIMARY>
| <PROPERTIES>
| <QUATERNARY>
| <QUERY>
- | <QUERYCOMMAND>
| <RANK>
- | <MODEL>
- | <RANKPROFILE>
- | <RANKPROPERTIES>
- | <RANKSCOREDROPLIMIT>
- | <RANKTYPE>
| <RAW>
| <REFERENCE>
- | <REMOVEIFZERO>
- | <RERANKCOUNT>
| <SCHEMA>
| <SEARCH>
| <SECONDARY>
- | <SECONDPHASE>
+ | <SEQUENTIAL>
| <SORTING>
| <SOURCE>
- | <PAGED>
| <SSCONTEXTUAL>
| <SSOVERRIDE>
| <SSTITLE>
@@ -2690,7 +2736,6 @@ String identifier() : { }
| <SUBSTRING>
| <SUFFIX>
| <SUMMARY>
- | <SUMMARYTO>
| <SYMMETRIC>
| <TERTIARY>
| <TEXT>
@@ -2700,7 +2745,6 @@ String identifier() : { }
| <UCA>
| <UNCASED>
| <URI>
- | <UPPERBOUND>
| <VARIABLE>
| <WEIGHT>
| <WEIGHTEDSET>
diff --git a/config-model/src/test/derived/globalphase_onnx_inside/onnx-models.cfg b/config-model/src/test/derived/globalphase_onnx_inside/onnx-models.cfg
index f705f896540..d63e85e2f19 100644
--- a/config-model/src/test/derived/globalphase_onnx_inside/onnx-models.cfg
+++ b/config-model/src/test/derived/globalphase_onnx_inside/onnx-models.cfg
@@ -25,7 +25,7 @@ model[].input[].source "rankingExpression(indirect_x)"
model[].output[].name "vector_Y"
model[].output[].as "foobar"
model[].dry_run_on_setup true
-model[].stateless_execution_mode ""
+model[].stateless_execution_mode "parallel"
model[].stateless_interop_threads 5
model[].stateless_intraop_threads 3
model[].gpu_device 2
diff --git a/config-model/src/test/derived/globalphase_onnx_inside/test.sd b/config-model/src/test/derived/globalphase_onnx_inside/test.sd
index cd3dfcea3d0..f7749e648e0 100644
--- a/config-model/src/test/derived/globalphase_onnx_inside/test.sd
+++ b/config-model/src/test/derived/globalphase_onnx_inside/test.sd
@@ -52,6 +52,7 @@ schema test {
intraop-threads: 3
interop-threads: 5
gpu-device: 2
+ execution-mode: parallel
}
first-phase {
expression: sum(attribute(aa))
diff --git a/config-model/src/test/java/com/yahoo/vespa/model/content/ContentClusterTest.java b/config-model/src/test/java/com/yahoo/vespa/model/content/ContentClusterTest.java
index b2175014cd4..488ad9f8484 100644
--- a/config-model/src/test/java/com/yahoo/vespa/model/content/ContentClusterTest.java
+++ b/config-model/src/test/java/com/yahoo/vespa/model/content/ContentClusterTest.java
@@ -1099,6 +1099,30 @@ public class ContentClusterTest extends ContentBaseTest {
}
}
+ private void verifySummaryDecodeType(String policy, DispatchConfig.SummaryDecodePolicy.Enum expected) {
+ TestProperties properties = new TestProperties();
+ if (policy != null) {
+ properties.setSummaryDecodePolicy(policy);
+ }
+ VespaModel model = createEnd2EndOneNode(properties);
+
+ ContentCluster cc = model.getContentClusters().get("storage");
+ DispatchConfig.Builder builder = new DispatchConfig.Builder();
+ cc.getSearch().getConfig(builder);
+
+ DispatchConfig cfg = new DispatchConfig(builder);
+ assertEquals(expected, cfg.summaryDecodePolicy());
+ }
+
+ @Test
+ public void verify_summary_decoding_controlled_by_properties() {
+ verifySummaryDecodeType(null, DispatchConfig.SummaryDecodePolicy.EAGER);
+ verifySummaryDecodeType("illegal-config", DispatchConfig.SummaryDecodePolicy.EAGER);
+ verifySummaryDecodeType("eager", DispatchConfig.SummaryDecodePolicy.EAGER);
+ verifySummaryDecodeType("ondemand", DispatchConfig.SummaryDecodePolicy.ONDEMAND);
+ verifySummaryDecodeType("on-demand", DispatchConfig.SummaryDecodePolicy.ONDEMAND);
+ }
+
private int resolveMaxCompactBuffers(OptionalInt maxCompactBuffers) {
TestProperties testProperties = new TestProperties();
if (maxCompactBuffers.isPresent()) {
diff --git a/config-provisioning/src/main/java/com/yahoo/config/provision/zone/ZoneApi.java b/config-provisioning/src/main/java/com/yahoo/config/provision/zone/ZoneApi.java
index b50d8ce2e7c..3e27cb4b6f6 100644
--- a/config-provisioning/src/main/java/com/yahoo/config/provision/zone/ZoneApi.java
+++ b/config-provisioning/src/main/java/com/yahoo/config/provision/zone/ZoneApi.java
@@ -43,4 +43,7 @@ public interface ZoneApi {
/** Returns the region name within the cloud, e.g. 'us-east-1' in AWS */
String getCloudNativeRegionName();
+ /** Returns the availability zone within the cloud, e.g. 'use1-az2' in AWS */
+ default String getCloudNativeAvailabilityZone() { throw new UnsupportedOperationException(); }
+
}
diff --git a/config/src/vespa/config/frt/frtconnection.cpp b/config/src/vespa/config/frt/frtconnection.cpp
index ff2a82f855b..4f1bf1c280c 100644
--- a/config/src/vespa/config/frt/frtconnection.cpp
+++ b/config/src/vespa/config/frt/frtconnection.cpp
@@ -104,7 +104,7 @@ void FRTConnection::calculateSuspension(ErrorType type)
switch(type) {
case TRANSIENT:
delay = std::min(MAX_DELAY_MULTIPLIER, ++_transientFailures) * _transientDelay;
- LOG(warning, "Connection to %s failed or timed out", _address.c_str());
+ LOG(debug, "Connection to %s failed or timed out", _address.c_str());
break;
case FATAL:
delay = std::min(MAX_DELAY_MULTIPLIER, ++_fatalFailures) * _fatalDelay;
@@ -112,7 +112,7 @@ void FRTConnection::calculateSuspension(ErrorType type)
}
_suspendedUntil = now + delay;
if (_suspendWarned < (now - WARN_INTERVAL)) {
- LOG(warning, "FRT Connection %s suspended until %s", _address.c_str(), vespalib::to_string(to_utc(_suspendedUntil)).c_str());
+ LOG(debug, "FRT Connection %s suspended until %s", _address.c_str(), vespalib::to_string(to_utc(_suspendedUntil)).c_str());
_suspendWarned = now;
}
}
diff --git a/configdefinitions/src/vespa/dispatch.def b/configdefinitions/src/vespa/dispatch.def
index 936b1400053..01fc5d48dfa 100644
--- a/configdefinitions/src/vespa/dispatch.def
+++ b/configdefinitions/src/vespa/dispatch.def
@@ -3,11 +3,11 @@
namespace=vespa.config.search
-# The active docs a group must have as a % of the average active docs of all other groups,
-# for that group to be included in queries
+## The active docs a group must have as a % of the average active docs of all other groups,
+## for that group to be included in queries
minActivedocsPercentage double default=97.0
-# Distribution policy for group selection
+## Distribution policy for group selection
distributionPolicy enum { ROUNDROBIN, BEST_OF_RANDOM_2, LATENCY_AMORTIZED_OVER_REQUESTS, LATENCY_AMORTIZED_OVER_TIME, ADAPTIVE } default=ADAPTIVE
## Maximum number of hits that will be requested from a single node
@@ -26,24 +26,28 @@ maxHitsPerNode int default=2147483647
## a setting of 1.0. This is a significant optimisation with with very little loss in presicion.
topKProbability double default=0.9999
-# Number of document replicas _per group_ that will be present in a stable cluster.
-# Should always be >= searchableCopies.
+## Number of document replicas _per group_ that will be present in a stable cluster.
+## Should always be >= searchableCopies.
redundancy long default=1
-# Minimum search coverage required before returning the results of a query
+## Minimum search coverage required before returning the results of a query
minSearchCoverage double default=100
-# Minimum wait time for full coverage after minimum coverage is achieved, factored based on time left at minimum coverage
+## Minimum wait time for full coverage after minimum coverage is achieved, factored based on time left at minimum coverage
minWaitAfterCoverageFactor double default=0
-# Maximum wait time for full coverage after minimum coverage is achieved, factored based on time left at minimum coverage
+## Maximum wait time for full coverage after minimum coverage is achieved, factored based on time left at minimum coverage
maxWaitAfterCoverageFactor double default=1
-# Number of JRT transport threads
+## Number of JRT transport threads
numJrtTransportThreads int default=8
-# Number of JRT connections per backend node
+## Number of JRT connections per backend node
numJrtConnectionsPerNode int default=8
-# Number of seconds to spend warming up code to prevent JIT cold start issues.
+## Number of seconds to spend warming up code to prevent JIT cold start issues.
warmuptime double default=0.1
+
+## Specifies how summary data are decoded
+## Eager will build java objects immediately, while ONDEMAND will do so when it is needed
+summaryDecodePolicy enum {EAGER, ONDEMAND} default=EAGER
diff --git a/configdefinitions/src/vespa/stor-distribution.def b/configdefinitions/src/vespa/stor-distribution.def
index 64c1316d46b..c5599a8589b 100644
--- a/configdefinitions/src/vespa/stor-distribution.def
+++ b/configdefinitions/src/vespa/stor-distribution.def
@@ -23,7 +23,7 @@ ready_copies int default=0
## If this option is set true, the distributor will try to enforce one active copy
## of buckets per leaf hierarchical group. This is a simple implementation for
## search to be able to setup top level dispatcher to only send search to all
-## nodes in one group as they have a static cost per node used. It used,
+## nodes in one group as they have a static cost per node used. If used,
## hierarchical grouping can not be used for other purposes. Using this option
## implies that:
## - ready_copies == redundancy
@@ -36,6 +36,7 @@ active_per_leaf_group bool default=false
## these cases they use many small groups to be able to scale. In such cases,
## where groups are small, having distributors able to take over when none are
## available in a group is useful.
+# TODO: Deprecated and unused, remove in Vespa 9
distributor_auto_ownership_transfer_on_whole_group_down bool default=true
## Hierarchical grouping divides the nodes into a tree of groups. Due to config
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ModelContextImpl.java b/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ModelContextImpl.java
index ad5423f0a94..7a2377594a1 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ModelContextImpl.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/deploy/ModelContextImpl.java
@@ -203,6 +203,7 @@ public class ModelContextImpl implements ModelContext {
private final boolean useRestrictedDataPlaneBindings;
private final int heapPercentage;
private final boolean enableGlobalPhase;
+ private final String summaryDecodePolicy;
public FeatureFlags(FlagSource source, ApplicationId appId, Version version) {
this.defaultTermwiseLimit = flagValue(source, appId, version, Flags.DEFAULT_TERM_WISE_LIMIT);
@@ -248,11 +249,13 @@ public class ModelContextImpl implements ModelContext {
this.useRestrictedDataPlaneBindings = flagValue(source, appId, version, Flags.RESTRICT_DATA_PLANE_BINDINGS);
this.heapPercentage = flagValue(source, appId, version, PermanentFlags.HEAP_SIZE_PERCENTAGE);
this.enableGlobalPhase = flagValue(source, appId, version, Flags.ENABLE_GLOBAL_PHASE);
+ this.summaryDecodePolicy = flagValue(source, appId, version, Flags.SUMMARY_DECODE_POLICY);
}
@Override public int heapSizePercentage() { return heapPercentage; }
@Override public String queryDispatchPolicy() { return queryDispatchPolicy; }
@Override public double queryDispatchWarmup() { return queryDispatchWarmup; }
+ @Override public String summaryDecodePolicy() { return summaryDecodePolicy; }
@Override public double defaultTermwiseLimit() { return defaultTermwiseLimit; }
@Override public String feedSequencerType() { return feedSequencer; }
@Override public String responseSequencerType() { return responseSequencer; }
diff --git a/container-core/src/main/java/com/yahoo/processing/Request.java b/container-core/src/main/java/com/yahoo/processing/Request.java
index cff546f1fd4..313df855cd4 100644
--- a/container-core/src/main/java/com/yahoo/processing/Request.java
+++ b/container-core/src/main/java/com/yahoo/processing/Request.java
@@ -30,13 +30,13 @@ public class Request extends FreezableClass implements Cloneable {
* The name of the chain of Processor instances which will be invoked when
* executing a request.
*/
- public static final CompoundName CHAIN = new CompoundName("chain");
+ public static final CompoundName CHAIN = CompoundName.from("chain");
/**
* The name of the request property used in the processing framework to
* store the incoming JDisc request.
*/
- public static final CompoundName JDISC_REQUEST = new CompoundName("jdisc.request");
+ public static final CompoundName JDISC_REQUEST = CompoundName.from("jdisc.request");
/**
* Creates a request with no properties
diff --git a/container-core/src/main/java/com/yahoo/processing/Response.java b/container-core/src/main/java/com/yahoo/processing/Response.java
index 59533900a0c..9f87f42ba65 100644
--- a/container-core/src/main/java/com/yahoo/processing/Response.java
+++ b/container-core/src/main/java/com/yahoo/processing/Response.java
@@ -35,7 +35,7 @@ import java.util.concurrent.TimeoutException;
*/
public class Response extends ListenableFreezableClass {
- private final static CompoundName freezeListenerKey =new CompoundName("processing.freezeListener");
+ private final static CompoundName freezeListenerKey = CompoundName.from("processing.freezeListener");
private final DataList<?> data;
diff --git a/container-core/src/main/java/com/yahoo/processing/handler/AbstractProcessingHandler.java b/container-core/src/main/java/com/yahoo/processing/handler/AbstractProcessingHandler.java
index f7aea8abbd1..837f356d4d9 100644
--- a/container-core/src/main/java/com/yahoo/processing/handler/AbstractProcessingHandler.java
+++ b/container-core/src/main/java/com/yahoo/processing/handler/AbstractProcessingHandler.java
@@ -46,7 +46,7 @@ import static com.yahoo.component.chain.ChainsConfigurer.prepareChainRegistry;
@SuppressWarnings("deprecation") // super class is deprecated
public abstract class AbstractProcessingHandler<COMPONENT extends Processor> extends LoggingRequestHandler {
- private final static CompoundName freezeListenerKey =new CompoundName("processing.freezeListener");
+ private final static CompoundName freezeListenerKey = CompoundName.from("processing.freezeListener");
public final static String DEFAULT_RENDERER_ID = "default";
@@ -112,7 +112,7 @@ public abstract class AbstractProcessingHandler<COMPONENT extends Processor> ext
public HttpResponse handle(HttpRequest request, ContentChannel channel) {
com.yahoo.processing.Request processingRequest = new com.yahoo.processing.Request();
populate("", request.propertyMap(), processingRequest.properties());
- populate("context", request.getJDiscRequest().context(), processingRequest.properties());
+ populate("context.", request.getJDiscRequest().context(), processingRequest.properties());
processingRequest.properties().set(Request.JDISC_REQUEST, request);
FreezeListener freezeListener = new FreezeListener(processingRequest, renderers, defaultRenderer, channel, renderingExecutor);
@@ -183,10 +183,9 @@ public abstract class AbstractProcessingHandler<COMPONENT extends Processor> ext
return properties.getString(Request.CHAIN,"default");
}
- private void populate(String prefixName,Map<String,?> parameters,Properties properties) {
- CompoundName prefix = new CompoundName(prefixName);
+ private void populate(String prefixName, Map<String,?> parameters,Properties properties) {
for (Map.Entry<String,?> entry : parameters.entrySet())
- properties.set(prefix.append(entry.getKey()), entry.getValue());
+ properties.set(CompoundName.from(prefixName + entry.getKey()), entry.getValue());
}
private static class FreezeListener implements Runnable, ResponseReceiver {
diff --git a/container-core/src/main/java/com/yahoo/processing/request/CompoundName.java b/container-core/src/main/java/com/yahoo/processing/request/CompoundName.java
index 66750b2943d..0edff9162b5 100644
--- a/container-core/src/main/java/com/yahoo/processing/request/CompoundName.java
+++ b/container-core/src/main/java/com/yahoo/processing/request/CompoundName.java
@@ -1,10 +1,13 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.processing.request;
+import com.yahoo.concurrent.CopyOnWriteHashMap;
+
import java.util.AbstractList;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
+import java.util.Map;
import static com.yahoo.text.Lowercase.toLowerCase;
@@ -18,9 +21,12 @@ import static com.yahoo.text.Lowercase.toLowerCase;
*/
public final class CompoundName {
- /**
- * The string name of this compound.
- */
+ private static final int MAX_CACHE_SIZE = 10_000;
+ private static final Map<String, CompoundName> cache = new CopyOnWriteHashMap<>();
+ /** The empty compound */
+ public static final CompoundName empty = CompoundName.from("");
+
+ /* The string name of this compound. */
private final String name;
private final String lowerCasedName;
@@ -34,16 +40,16 @@ public final class CompoundName {
/** This name with the last component removed */
private final CompoundName first;
- /** The empty compound */
- public static final CompoundName empty = new CompoundName("");
-
/**
* Constructs this from a string which may contains dot-separated components
*
* @throws NullPointerException if name is null
*/
public CompoundName(String name) {
- this(name, parse(name).toArray(new String[1]));
+ this(name, false);
+ }
+ private CompoundName(String name, boolean useCache) {
+ this(name, parse(name).toArray(new String[0]), useCache);
}
/** Constructs this from an array of name components which are assumed not to contain dots */
@@ -57,7 +63,7 @@ public final class CompoundName {
}
private CompoundName(String [] compounds) {
- this(toCompoundString(compounds), compounds);
+ this(toCompoundString(compounds), compounds, false);
}
/**
@@ -67,7 +73,7 @@ public final class CompoundName {
* @param name the string representation of the compounds
* @param compounds the compounds of this name
*/
- private CompoundName(String name, String [] compounds) {
+ private CompoundName(String name, String [] compounds, boolean useCache) {
if (name == null) throw new NullPointerException("Name can not be null");
this.name = name;
@@ -82,12 +88,27 @@ public final class CompoundName {
this.compounds = new ImmutableArrayList(compounds);
this.hashCode = this.compounds.hashCode();
- rest = (compounds.length > 1)
- ? new CompoundName(name.substring(compounds[0].length()+1), Arrays.copyOfRange(compounds, 1, compounds.length))
- : empty;
- first = (compounds.length > 1)
- ? new CompoundName(name.substring(0, name.length() - (compounds[compounds.length-1].length()+1)), Arrays.copyOfRange(compounds, 0, compounds.length-1))
- : empty;
+ if (compounds.length > 1) {
+ String restName = name.substring(compounds[0].length()+1);
+ if (useCache) {
+ rest = cache.computeIfAbsent(restName, (key) -> new CompoundName(key, Arrays.copyOfRange(compounds, 1, compounds.length), useCache));
+ } else {
+ rest = new CompoundName(restName, Arrays.copyOfRange(compounds, 1, compounds.length), useCache);
+ }
+ } else {
+ rest = empty;
+ }
+
+ if (compounds.length > 1) {
+ String firstName = name.substring(0, name.length() - (compounds[compounds.length-1].length()+1));
+ if (useCache) {
+ first = cache.computeIfAbsent(firstName, (key) -> new CompoundName(key, Arrays.copyOfRange(compounds, 0, compounds.length-1), useCache));
+ } else {
+ first = new CompoundName(firstName, Arrays.copyOfRange(compounds, 0, compounds.length-1), useCache);
+ }
+ } else {
+ first = empty;
+ }
}
private static List<String> parse(String s) {
@@ -134,7 +155,7 @@ public final class CompoundName {
int count = 0;
for (String s : compounds) { newCompounds[count++] = s; }
for (String s : name.compounds) { newCompounds[count++] = s; }
- return new CompoundName(concat(this.name, name.name), newCompounds);
+ return new CompoundName(concat(this.name, name.name), newCompounds, false);
}
private static String concat(String name1, String name2) {
@@ -298,7 +319,21 @@ public final class CompoundName {
return b.length()==0 ? "" : b.substring(0, b.length()-1);
}
- public static CompoundName from(String name) { return new CompoundName(name); }
+ /**
+ * Creates a CompoundName from a string, possibly reusing from cache.
+ * Prefer over constructing on the fly.
+ **/
+ public static CompoundName from(String name) {
+ CompoundName found = cache.get(name);
+ if (found != null) return found;
+
+ if (cache.size() < MAX_CACHE_SIZE) {
+ CompoundName compound = new CompoundName(name, true);
+ cache.put(name, compound);
+ return compound;
+ }
+ return new CompoundName(name, false);
+ }
private static class ImmutableArrayList extends AbstractList<String> {
diff --git a/container-core/src/test/java/com/yahoo/processing/request/CompoundNameTestCase.java b/container-core/src/test/java/com/yahoo/processing/request/CompoundNameTestCase.java
index 4bbece0af29..eccc4dd8842 100644
--- a/container-core/src/test/java/com/yahoo/processing/request/CompoundNameTestCase.java
+++ b/container-core/src/test/java/com/yahoo/processing/request/CompoundNameTestCase.java
@@ -18,59 +18,61 @@ import static org.junit.jupiter.api.Assertions.*;
public class CompoundNameTestCase {
private static final String NAME = "com.yahoo.processing.request.CompoundNameTestCase";
- private final CompoundName cn = new CompoundName(NAME);
+ private static final CompoundName C_NAME = CompoundName.from(NAME);
+ private static final CompoundName C_abcde = CompoundName.from("a.b.c.d.e");
void verifyStrict(CompoundName expected, CompoundName actual) {
assertEquals(expected, actual);
assertEquals(expected.asList(), actual.asList());
}
void verifyStrict(String expected, CompoundName actual) {
- verifyStrict(new CompoundName(expected), actual);
+ verifyStrict(CompoundName.from(expected), actual);
}
@Test
final void testLast() {
- assertEquals(NAME.substring(NAME.lastIndexOf('.') + 1), cn.last());
+ assertEquals(NAME.substring(NAME.lastIndexOf('.') + 1), C_NAME.last());
}
@Test
final void testFirst() {
- assertEquals(NAME.substring(0, NAME.indexOf('.')), cn.first());
+ assertEquals(NAME.substring(0, NAME.indexOf('.')), C_NAME.first());
}
@Test
final void testRest() {
- verifyStrict(NAME.substring(NAME.indexOf('.') + 1), cn.rest());
+ verifyStrict(NAME.substring(NAME.indexOf('.') + 1), C_NAME.rest());
}
@Test
final void testRestN() {
- verifyStrict("a.b.c.d.e", new CompoundName("a.b.c.d.e").rest(0));
- verifyStrict("b.c.d.e", new CompoundName("a.b.c.d.e").rest(1));
- verifyStrict("c.d.e", new CompoundName("a.b.c.d.e").rest(2));
- verifyStrict("d.e", new CompoundName("a.b.c.d.e").rest(3));
- verifyStrict("e", new CompoundName("a.b.c.d.e").rest(4));
- verifyStrict(CompoundName.empty, new CompoundName("a.b.c.d.e").rest(5));
+ verifyStrict("a.b.c.d.e", C_abcde.rest(0));
+ verifyStrict("b.c.d.e", C_abcde.rest(1));
+ verifyStrict("c.d.e", C_abcde.rest(2));
+ verifyStrict("d.e", C_abcde.rest(3));
+ verifyStrict("e", C_abcde.rest(4));
+ verifyStrict(CompoundName.empty, C_abcde.rest(5));
}
@Test
final void testFirstN() {
- verifyStrict("a.b.c.d.e", new CompoundName("a.b.c.d.e").first(5));
- verifyStrict("a.b.c.d", new CompoundName("a.b.c.d.e").first(4));
- verifyStrict("a.b.c", new CompoundName("a.b.c.d.e").first(3));
- verifyStrict("a.b", new CompoundName("a.b.c.d.e").first(2));
- verifyStrict("a", new CompoundName("a.b.c.d.e").first(1));
- verifyStrict(CompoundName.empty, new CompoundName("a.b.c.d.e").first(0));
+ verifyStrict("a.b.c.d.e", C_abcde.first(5));
+ verifyStrict("a.b.c.d", C_abcde.first(4));
+ verifyStrict("a.b.c", C_abcde.first(3));
+ verifyStrict("a.b", C_abcde.first(2));
+ verifyStrict("a", C_abcde.first(1));
+ verifyStrict(CompoundName.empty, C_abcde.first(0));
}
@Test
final void testPrefix() {
- assertTrue(new CompoundName("a.b.c").hasPrefix(new CompoundName("")));
- assertTrue(new CompoundName("a.b.c").hasPrefix(new CompoundName("a")));
- assertTrue(new CompoundName("a.b.c").hasPrefix(new CompoundName("a.b")));
- assertTrue(new CompoundName("a.b.c").hasPrefix(new CompoundName("a.b.c")));
+ CompoundName abc = CompoundName.from("a.b.c");
+ assertTrue(abc.hasPrefix(CompoundName.empty));
+ assertTrue(abc.hasPrefix(CompoundName.from("a")));
+ assertTrue(abc.hasPrefix(CompoundName.from("a.b")));
+ assertTrue(abc.hasPrefix(CompoundName.from("a.b.c")));
- assertFalse(new CompoundName("a.b.c").hasPrefix(new CompoundName("a.b.c.d")));
- assertFalse(new CompoundName("a.b.c").hasPrefix(new CompoundName("a.b.d")));
+ assertFalse(abc.hasPrefix(CompoundName.from("a.b.c.d")));
+ assertFalse(abc.hasPrefix(CompoundName.from("a.b.d")));
}
@Test
@@ -87,28 +89,28 @@ public class CompoundNameTestCase {
for (@SuppressWarnings("unused") String x : i) {
++n;
}
- assertEquals(n, cn.size());
+ assertEquals(n, C_NAME.size());
}
@Test
final void testGet() {
- String s = cn.get(0);
+ String s = C_NAME.get(0);
assertEquals(NAME.substring(0, NAME.indexOf('.')), s);
}
@Test
final void testIsCompound() {
- assertTrue(cn.isCompound());
+ assertTrue(C_NAME.isCompound());
}
@Test
final void testIsEmpty() {
- assertFalse(cn.isEmpty());
+ assertFalse(C_NAME.isEmpty());
}
@Test
final void testAsList() {
- List<String> l = cn.asList();
+ List<String> l = C_NAME.asList();
Splitter peoplesFront = Splitter.on('.');
Iterable<String> answer = peoplesFront.split(NAME);
Iterator<String> expected = answer.iterator();
@@ -120,35 +122,35 @@ public class CompoundNameTestCase {
@Test
final void testEqualsObject() {
- assertNotEquals(cn, NAME);
- assertNotEquals(cn, null);
- verifyStrict(cn, cn);
- verifyStrict(cn, new CompoundName(NAME));
+ assertNotEquals(C_NAME, NAME);
+ assertNotEquals(C_NAME, null);
+ verifyStrict(C_NAME, C_NAME);
+ verifyStrict(C_NAME, CompoundName.from(NAME));
}
@Test
final void testEmptyNonEmpty() {
- assertTrue(new CompoundName("").isEmpty());
- assertEquals(0, new CompoundName("").size());
- assertFalse(new CompoundName("a").isEmpty());
- assertEquals(1, new CompoundName("a").size());
- CompoundName empty = new CompoundName("a.b.c");
+ assertTrue(CompoundName.empty.isEmpty());
+ assertEquals(0, CompoundName.empty.size());
+ assertFalse(CompoundName.from("a").isEmpty());
+ assertEquals(1, CompoundName.from("a").size());
+ CompoundName empty = CompoundName.from("a.b.c");
verifyStrict(empty, empty.rest(0));
assertNotEquals(empty, empty.rest(1));
}
@Test
final void testGetLowerCasedName() {
- assertEquals(Lowercase.toLowerCase(NAME), cn.getLowerCasedName());
+ assertEquals(Lowercase.toLowerCase(NAME), C_NAME.getLowerCasedName());
}
@Test
void testAppendCompound() {
- verifyStrict("a.b.c.d", new CompoundName("").append(new CompoundName("a.b.c.d")));
- verifyStrict("a.b.c.d", new CompoundName("a").append(new CompoundName("b.c.d")));
- verifyStrict("a.b.c.d", new CompoundName("a.b").append(new CompoundName("c.d")));
- verifyStrict("a.b.c.d", new CompoundName("a.b.c").append(new CompoundName("d")));
- verifyStrict("a.b.c.d", new CompoundName("a.b.c.d").append(new CompoundName("")));
+ verifyStrict("a.b.c.d", CompoundName.empty.append(CompoundName.from("a.b.c.d")));
+ verifyStrict("a.b.c.d", CompoundName.from("a").append(CompoundName.from("b.c.d")));
+ verifyStrict("a.b.c.d", CompoundName.from("a.b").append(CompoundName.from("c.d")));
+ verifyStrict("a.b.c.d", CompoundName.from("a.b.c").append(CompoundName.from("d")));
+ verifyStrict("a.b.c.d", CompoundName.from("a.b.c.d").append(CompoundName.empty));
}
@Test
@@ -156,13 +158,13 @@ public class CompoundNameTestCase {
CompoundName empty = new CompoundName("");
assertTrue(empty.hasPrefix(empty));
- assertTrue(new CompoundName("a").hasPrefix(empty));
+ assertTrue(CompoundName.from("a").hasPrefix(empty));
}
@Test
void whole_components_must_match_to_be_prefix() {
- CompoundName stringPrefix = new CompoundName("a");
- CompoundName name = new CompoundName("aa");
+ CompoundName stringPrefix = CompoundName.from("a");
+ CompoundName name = CompoundName.from("aa");
assertFalse(name.hasPrefix(stringPrefix));
}
@@ -171,7 +173,7 @@ public class CompoundNameTestCase {
void testFirstRest() {
verifyStrict(CompoundName.empty, CompoundName.empty.rest());
- CompoundName n = new CompoundName("on.two.three");
+ CompoundName n = CompoundName.from("on.two.three");
assertEquals("on", n.first());
verifyStrict("two.three", n.rest());
n = n.rest();
@@ -190,7 +192,7 @@ public class CompoundNameTestCase {
@Test
void testHashCodeAndEquals() {
- CompoundName n1 = new CompoundName("venn.d.a");
+ CompoundName n1 = CompoundName.from("venn.d.a");
CompoundName n2 = new CompoundName(n1.asList());
assertEquals(n1.hashCode(), n2.hashCode());
verifyStrict(n1, n2);
@@ -198,12 +200,12 @@ public class CompoundNameTestCase {
@Test
void testAppendString() {
- verifyStrict("a", new CompoundName("a").append(""));
- verifyStrict("a", new CompoundName("").append("a"));
- verifyStrict("a.b", new CompoundName("a").append("b"));
- verifyStrict("a.b.c.d", new CompoundName("a.b").append("c.d"));
+ verifyStrict("a", CompoundName.from("a").append(""));
+ verifyStrict("a", CompoundName.empty.append("a"));
+ verifyStrict("a.b", CompoundName.from("a").append("b"));
+ verifyStrict("a.b.c.d", CompoundName.from("a.b").append("c.d"));
- CompoundName name = new CompoundName("a.b");
+ CompoundName name = CompoundName.from("a.b");
verifyStrict("a.b.c", name.append("c"));
verifyStrict("a.b.d", name.append("d"));
verifyStrict("a.b.d.e", name.append("d.e"));
@@ -218,7 +220,7 @@ public class CompoundNameTestCase {
@Test
void testAsList2() {
- assertEquals("[one]", new CompoundName("one").asList().toString());
- assertEquals("[one, two, three]", new CompoundName("one.two.three").asList().toString());
+ assertEquals("[one]", CompoundName.from("one").asList().toString());
+ assertEquals("[one, two, three]", CompoundName.from("one.two.three").asList().toString());
}
}
diff --git a/container-core/src/test/java/com/yahoo/processing/request/test/RequestTestCase.java b/container-core/src/test/java/com/yahoo/processing/request/test/RequestTestCase.java
index c82b85d84d9..8e960bd1305 100644
--- a/container-core/src/test/java/com/yahoo/processing/request/test/RequestTestCase.java
+++ b/container-core/src/test/java/com/yahoo/processing/request/test/RequestTestCase.java
@@ -9,7 +9,9 @@ import com.yahoo.processing.request.properties.PropertyMap;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Tests using requests
@@ -19,9 +21,26 @@ import static org.junit.jupiter.api.Assertions.assertNull;
public class RequestTestCase {
private static final double delta = 0.0000000001;
+ private static final CompoundName C_a = CompoundName.from("a");
+
+ private static final CompoundName C_B = CompoundName.from("b");
+ private static final CompoundName C_C = CompoundName.from("c");
+ private static final CompoundName C_D = CompoundName.from("d");
+ private static final CompoundName C_f = CompoundName.from("f");
+ private static final CompoundName C_g = CompoundName.from("g");
+ private static final CompoundName C_I = CompoundName.from("i");
+ private static final CompoundName C_L = CompoundName.from("l");
+ private static final CompoundName C_M = CompoundName.from("m");
+ private static final CompoundName C_N = CompoundName.from("n");
+ private static final CompoundName C_o = CompoundName.from("o");
+ private static final CompoundName C_x = CompoundName.from("x");
+ private static final CompoundName C_none = CompoundName.from("none");
+
+
@Test
void testProperties() {
+
Properties p = new PropertyMap();
p.set("a", "a1");
Request r = new Request(p);
@@ -33,38 +52,38 @@ public class RequestTestCase {
assertEquals("b1", r.properties().get("b", "default"));
assertEquals("default", r.properties().get("c", "default"));
assertNull(r.properties().get("c"));
- assertEquals("b1", r.properties().get(new CompoundName("b")));
- assertEquals("b1", r.properties().get(new CompoundName("b"), "default"));
- assertEquals("default", r.properties().get(new CompoundName("c"), "default"));
- assertNull(r.properties().get(new CompoundName("c")));
+ assertEquals("b1", r.properties().get(C_B));
+ assertEquals("b1", r.properties().get(C_B, "default"));
+ assertEquals("default", r.properties().get(C_C, "default"));
+ assertNull(r.properties().get(C_C));
assertEquals("b1", r.properties().getString("b"));
assertEquals("b1", r.properties().getString("b", "default"));
assertEquals("default", r.properties().getString("c", "default"));
assertNull(r.properties().getString("c"));
- assertEquals("b1", r.properties().getString(new CompoundName("b")));
- assertEquals("b1", r.properties().getString(new CompoundName("b"), "default"));
- assertEquals("default", r.properties().getString(new CompoundName("c"), "default"));
- assertNull(r.properties().getString(new CompoundName("c")));
+ assertEquals("b1", r.properties().getString(C_B));
+ assertEquals("b1", r.properties().getString(C_B, "default"));
+ assertEquals("default", r.properties().getString(C_C, "default"));
+ assertNull(r.properties().getString(C_C));
r.properties().set("i", 7);
assertEquals(7, (int) r.properties().getInteger("i"));
assertEquals(7, (int) r.properties().getInteger("i", 3));
assertEquals(3, (int) r.properties().getInteger("n", 3));
assertNull(r.properties().getInteger("n"));
- assertEquals(7, (int) r.properties().getInteger(new CompoundName("i")));
- assertEquals(7, (int) r.properties().getInteger(new CompoundName("i"), 3));
- assertEquals(3, (int) r.properties().getInteger(new CompoundName("n"), 3));
+ assertEquals(7, (int) r.properties().getInteger(C_I));
+ assertEquals(7, (int) r.properties().getInteger(C_I, 3));
+ assertEquals(3, (int) r.properties().getInteger(C_N, 3));
assertNull(r.properties().getInteger("n"));
- r.properties().set(new CompoundName("l"), 7);
+ r.properties().set(C_L, 7);
assertEquals(7, (long) r.properties().getLong("l"));
- assertEquals(7, (long) r.properties().getLong("l", 3l));
- assertEquals(3, (long) r.properties().getLong("m", 3l));
+ assertEquals(7, (long) r.properties().getLong("l", 3L));
+ assertEquals(3, (long) r.properties().getLong("m", 3L));
assertNull(r.properties().getInteger("m"));
- assertEquals(7, (long) r.properties().getLong(new CompoundName("l")));
- assertEquals(7, (long) r.properties().getLong(new CompoundName("l"), 3l));
- assertEquals(3, (long) r.properties().getLong(new CompoundName("m"), 3l));
+ assertEquals(7, (long) r.properties().getLong(C_L));
+ assertEquals(7, (long) r.properties().getLong(C_L, 3L));
+ assertEquals(3, (long) r.properties().getLong(C_M, 3L));
assertNull(r.properties().getInteger("m"));
r.properties().set("d", 7.3);
@@ -72,29 +91,29 @@ public class RequestTestCase {
assertEquals(7.3, r.properties().getDouble("d", 3.4d), delta);
assertEquals(3.4, r.properties().getDouble("f", 3.4d), delta);
assertNull(r.properties().getDouble("f"));
- assertEquals(7.3, r.properties().getDouble(new CompoundName("d")), delta);
- assertEquals(7.3, r.properties().getDouble(new CompoundName("d"), 3.4d), delta);
- assertEquals(3.4, r.properties().getDouble(new CompoundName("f"), 3.4d), delta);
+ assertEquals(7.3, r.properties().getDouble(C_D), delta);
+ assertEquals(7.3, r.properties().getDouble(C_D, 3.4d), delta);
+ assertEquals(3.4, r.properties().getDouble(C_f, 3.4d), delta);
assertNull(r.properties().getDouble("f"));
r.properties().set("o", true);
- assertEquals(true, r.properties().getBoolean("o"));
- assertEquals(true, r.properties().getBoolean("o", true));
- assertEquals(true, r.properties().getBoolean("g", true));
- assertEquals(false, r.properties().getBoolean("g"));
- assertEquals(true, r.properties().getBoolean(new CompoundName("o")));
- assertEquals(true, r.properties().getBoolean(new CompoundName("o"), true));
- assertEquals(true, r.properties().getBoolean(new CompoundName("g"), true));
- assertEquals(false, r.properties().getBoolean("g"));
-
- r.properties().set(new CompoundName("x.y"), "x1.y1");
+ assertTrue(r.properties().getBoolean("o"));
+ assertTrue(r.properties().getBoolean("o", true));
+ assertTrue(r.properties().getBoolean("g", true));
+ assertFalse(r.properties().getBoolean("g"));
+ assertTrue(r.properties().getBoolean(C_o));
+ assertTrue(r.properties().getBoolean(C_o, true));
+ assertTrue(r.properties().getBoolean(C_g, true));
+ assertFalse(r.properties().getBoolean("g"));
+
+ r.properties().set(CompoundName.from("x.y"), "x1.y1");
r.properties().set("x.z", "x1.z1");
assertEquals(8, r.properties().listProperties().size());
assertEquals(0, r.properties().listProperties("a").size());
- assertEquals(0, r.properties().listProperties(new CompoundName("a")).size());
- assertEquals(0, r.properties().listProperties(new CompoundName("none")).size());
- assertEquals(2, r.properties().listProperties(new CompoundName("x")).size());
+ assertEquals(0, r.properties().listProperties(C_a).size());
+ assertEquals(0, r.properties().listProperties(C_none).size());
+ assertEquals(2, r.properties().listProperties(C_x).size());
assertEquals(2, r.properties().listProperties("x").size());
}
diff --git a/container-search/src/main/java/com/yahoo/prelude/IndexFacts.java b/container-search/src/main/java/com/yahoo/prelude/IndexFacts.java
index 92ce6abb319..88a37ea5a02 100644
--- a/container-search/src/main/java/com/yahoo/prelude/IndexFacts.java
+++ b/container-search/src/main/java/com/yahoo/prelude/IndexFacts.java
@@ -6,11 +6,11 @@ import com.yahoo.search.Query;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.TreeSet;
import static com.yahoo.text.Lowercase.toLowerCase;
@@ -32,16 +32,6 @@ public class IndexFacts {
private Map<String, List<String>> clusterByDocument;
- private static class DocumentTypeListOffset {
- public final int offset;
- public final SearchDefinition searchDefinition;
-
- public DocumentTypeListOffset(int offset, SearchDefinition searchDefinition) {
- this.offset = offset;
- this.searchDefinition = searchDefinition;
- }
- }
-
/** A Map of all known search definitions indexed by name */
private Map<String, SearchDefinition> searchDefinitions = new LinkedHashMap<>();
@@ -110,34 +100,32 @@ public class IndexFacts {
private boolean isIndexFromDocumentTypes(String indexName, List<String> documentTypes) {
if ( ! isInitialized()) return true;
- if (documentTypes.isEmpty()) {
- return unionSearchDefinition.getIndex(indexName) != null;
- }
+ if (documentTypes.isEmpty()) return unionSearchDefinition.getIndex(indexName) != null;
- DocumentTypeListOffset sd = chooseSearchDefinition(documentTypes, 0);
- while (sd != null) {
- Index index = sd.searchDefinition.getIndex(indexName);
- if (index != null) {
- return true;
+ for (String docName : documentTypes) {
+ SearchDefinition sd = searchDefinitions.get(docName);
+ if (sd != null) {
+ Index index = sd.getIndex(indexName);
+ if (index != null) return true;
}
- sd = chooseSearchDefinition(documentTypes, sd.offset);
}
-
return false;
}
private String getCanonicNameFromDocumentTypes(String indexName, List<String> documentTypes) {
if (!isInitialized()) return indexName;
+ String lowerCased = toLowerCase(indexName);
if (documentTypes.isEmpty()) {
- Index index = unionSearchDefinition.getIndexByLowerCase(toLowerCase(indexName));
+ Index index = unionSearchDefinition.getIndexByLowerCase(lowerCased);
return index == null ? indexName : index.getName();
}
- DocumentTypeListOffset sd = chooseSearchDefinition(documentTypes, 0);
- while (sd != null) {
- Index index = sd.searchDefinition.getIndexByLowerCase(toLowerCase(indexName));
- if (index != null) return index.getName();
- sd = chooseSearchDefinition(documentTypes, sd.offset);
+ for (String docName : documentTypes) {
+ SearchDefinition sd = searchDefinitions.get(docName);
+ if (sd != null) {
+ Index index = sd.getIndexByLowerCase(lowerCased);
+ if (index != null) return index.getName();
+ }
}
return indexName;
}
@@ -158,13 +146,12 @@ public class IndexFacts {
return index;
}
- DocumentTypeListOffset sd = chooseSearchDefinition(documentTypes, 0);
-
- while (sd != null) {
- Index index = sd.searchDefinition.getIndex(canonicName);
-
- if (index != null) return index;
- sd = chooseSearchDefinition(documentTypes, sd.offset);
+ for (String docName : documentTypes) {
+ SearchDefinition sd = searchDefinitions.get(docName);
+ if (sd != null) {
+ Index index = sd.getIndex(canonicName);
+ if (index != null) return index;
+ }
}
return Index.nullIndex;
}
@@ -187,7 +174,7 @@ public class IndexFacts {
* Given a search list which is a mixture of document types and cluster
* names, and a restrict list which is a list of document types, return a
* set of all valid document types for this combination. Most use-cases for
- * fetching index settings will involve calling this method with the the
+ * fetching index settings will involve calling this method with the
* incoming query's {@link com.yahoo.search.query.Model#getSources()} and
* {@link com.yahoo.search.query.Model#getRestrict()} as input parameters
* before calling any other method of this class.
@@ -196,20 +183,20 @@ public class IndexFacts {
* @param restrict the restrict list for a query
* @return a (possibly empty) set of valid document types
*/
- private Set<String> resolveDocumentTypes(Collection<String> sources, Collection<String> restrict,
+ private Set<String> resolveDocumentTypes(Collection<String> sources, Set<String> restrict,
Set<String> candidateDocumentTypes) {
sources = emptyCollectionIfNull(sources);
- restrict = emptyCollectionIfNull(restrict);
+ restrict = emptySetIfNull(restrict);
if (sources.isEmpty()) {
if ( ! restrict.isEmpty()) {
- return new TreeSet<>(restrict);
+ return Set.copyOf(restrict);
} else {
return candidateDocumentTypes;
}
}
- Set<String> toSearch = new TreeSet<>();
+ Set<String> toSearch = new HashSet<>();
for (String source : sources) { // source: a document type or a cluster containing them
List<String> clusterDocTypes = clusters.get(source);
if (clusterDocTypes == null) { // source was a document type
@@ -235,21 +222,8 @@ public class IndexFacts {
private Collection<String> emptyCollectionIfNull(Collection<String> collection) {
return collection == null ? List.of() : collection;
}
-
- /**
- * Chooses the correct search definition, default if in doubt.
- *
- * @return the search definition to use
- */
- private DocumentTypeListOffset chooseSearchDefinition(List<String> documentTypes, int index) {
- while (index < documentTypes.size()) {
- String docName = documentTypes.get(index++);
- SearchDefinition sd = searchDefinitions.get(docName);
- if (sd != null) {
- return new DocumentTypeListOffset(index, sd);
- }
- }
- return null;
+ private Set<String> emptySetIfNull(Set<String> collection) {
+ return collection == null ? Set.of() : collection;
}
/**
@@ -279,10 +253,6 @@ public class IndexFacts {
return frozen;
}
- private void ensureNotFrozen() {
- if (frozen) throw new IllegalStateException("Tried to modify frozen IndexFacts instance.");
- }
-
public String getDefaultPosition(String sdName) {
SearchDefinition sd;
if (sdName == null) {
@@ -300,12 +270,16 @@ public class IndexFacts {
return new Session(query);
}
- public Session newSession(Collection<String> sources, Collection<String> restrict) {
+ public Session newSession() {
+ return new Session(Set.of(), Set.of());
+ }
+
+ public Session newSession(Collection<String> sources, Set<String> restrict) {
return new Session(sources, restrict);
}
public Session newSession(Collection<String> sources,
- Collection<String> restrict,
+ Set<String> restrict,
Set<String> candidateDocumentTypes) {
return new Session(sources, restrict, candidateDocumentTypes);
}
@@ -323,12 +297,12 @@ public class IndexFacts {
documentTypes = List.copyOf(resolveDocumentTypes(query));
}
- private Session(Collection<String> sources, Collection<String> restrict) {
+ private Session(Collection<String> sources, Set<String> restrict) {
// Assumption: Search definition name equals document name.
documentTypes = List.copyOf(resolveDocumentTypes(sources, restrict, searchDefinitions.keySet()));
}
- private Session(Collection<String> sources, Collection<String> restrict, Set<String> candidateDocumentTypes) {
+ private Session(Collection<String> sources, Set<String> restrict, Set<String> candidateDocumentTypes) {
documentTypes = List.copyOf(resolveDocumentTypes(sources, restrict, candidateDocumentTypes));
}
diff --git a/container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java b/container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java
index 08d8d54bc53..f0e3e3f3e44 100644
--- a/container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java
@@ -1,8 +1,8 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.prelude.cluster;
-import com.yahoo.component.annotation.Inject;
import com.yahoo.component.ComponentId;
+import com.yahoo.component.annotation.Inject;
import com.yahoo.component.chain.dependencies.After;
import com.yahoo.component.provider.ComponentRegistry;
import com.yahoo.container.QrSearchersConfig;
@@ -28,10 +28,6 @@ import com.yahoo.vespa.streamingvisitors.VdsStreamingSearcher;
import com.yahoo.yolean.Exceptions;
import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
import java.util.UUID;
@@ -57,16 +53,14 @@ public class ClusterSearcher extends Searcher {
private final String searchClusterName;
- // The set of document types contained in this search cluster
- private final Set<String> schemas;
+ private final SchemaResolver schemaResolver;
private final long maxQueryTimeout; // in milliseconds
private final long maxQueryCacheTimeout; // in milliseconds
private final VespaBackEndSearcher server;
private final Executor executor;
- private final GlobalPhaseRanker globalPhaseHelper;
- private final boolean enableGlobalPhase;
+ private final GlobalPhaseRanker globalPhaseRanker;
@Inject
public ClusterSearcher(ComponentId id,
@@ -76,16 +70,16 @@ public class ClusterSearcher extends Searcher {
DocumentdbInfoConfig documentDbConfig,
SchemaInfo schemaInfo,
ComponentRegistry<Dispatcher> dispatchers,
- GlobalPhaseRanker globalPhaseHelper,
+ GlobalPhaseRanker globalPhaseRanker,
VipStatus vipStatus,
VespaDocumentAccess access) {
super(id);
this.executor = executor;
- this.globalPhaseHelper = globalPhaseHelper;
int searchClusterIndex = clusterConfig.clusterId();
searchClusterName = clusterConfig.clusterName();
QrSearchersConfig.Searchcluster searchClusterConfig = getSearchClusterConfigFromClusterName(qrsConfig, searchClusterName);
- schemas = new LinkedHashSet<>();
+ this.globalPhaseRanker = searchClusterConfig.globalphase() ? globalPhaseRanker : null;
+ this.schemaResolver = new SchemaResolver(documentDbConfig);
maxQueryTimeout = ParameterParser.asMilliSeconds(clusterConfig.maxQueryTimeout(), DEFAULT_MAX_QUERY_TIMEOUT);
maxQueryCacheTimeout = ParameterParser.asMilliSeconds(clusterConfig.maxQueryCacheTimeout(), DEFAULT_MAX_QUERY_CACHE_TIMEOUT);
@@ -94,9 +88,6 @@ public class ClusterSearcher extends Searcher {
.com().yahoo().prelude().fastsearch().FastSearcher().docsum()
.defaultclass());
- for (DocumentdbInfoConfig.Documentdb docDb : documentDbConfig.documentdb())
- schemas.add(docDb.name());
-
String uniqueServerId = UUID.randomUUID().toString();
if (searchClusterConfig.indexingmode() == STREAMING) {
server = vdsCluster(uniqueServerId, searchClusterIndex,
@@ -106,7 +97,6 @@ public class ClusterSearcher extends Searcher {
server = searchDispatch(searchClusterIndex, searchClusterName, uniqueServerId,
docSumParams, documentDbConfig, schemaInfo, dispatchers);
}
- enableGlobalPhase = searchClusterConfig.globalphase();
}
private static QrSearchersConfig.Searchcluster getSearchClusterConfigFromClusterName(QrSearchersConfig config, String name) {
@@ -159,14 +149,13 @@ public class ClusterSearcher extends Searcher {
/** Do not use, for internal testing purposes only. **/
ClusterSearcher(Set<String> schemas, VespaBackEndSearcher searcher, Executor executor) {
- this.schemas = schemas;
+ this.schemaResolver = new SchemaResolver(schemas);
searchClusterName = "testScenario";
maxQueryTimeout = DEFAULT_MAX_QUERY_TIMEOUT;
maxQueryCacheTimeout = DEFAULT_MAX_QUERY_CACHE_TIMEOUT;
server = searcher;
this.executor = executor;
- this.globalPhaseHelper = null;
- this.enableGlobalPhase = false;
+ this.globalPhaseRanker = null;
}
/** Do not use, for internal testing purposes only. **/
@@ -232,8 +221,9 @@ public class ClusterSearcher extends Searcher {
}
private Result doSearch(Searcher searcher, Query query, Execution execution) {
+ var schemas = schemaResolver.resolve(query, execution);
if (schemas.size() > 1) {
- return searchMultipleDocumentTypes(searcher, query, execution);
+ return searchMultipleDocumentTypes(searcher, query, execution, schemas);
} else {
String docType = schemas.iterator().next();
query.getModel().setRestrict(docType);
@@ -247,10 +237,13 @@ public class ClusterSearcher extends Searcher {
throw new IllegalStateException("perSchemaSearch must always be called with 1 schema, got: " + restrict.size());
}
String schema = restrict.iterator().next();
- Result result = searcher.search(query, execution);
- if (globalPhaseHelper != null && enableGlobalPhase) {
- globalPhaseHelper.process(query, result, schema);
+ boolean useGlobalPhase = globalPhaseRanker != null;
+ if (useGlobalPhase) {
+ var error = globalPhaseRanker.validateNoSorting(query, schema).orElse(null);
+ if (error != null) return new Result(query, error);
}
+ Result result = searcher.search(query, execution);
+ if (useGlobalPhase) globalPhaseRanker.rerankHits(query, result, schema);
return result;
}
@@ -266,8 +259,7 @@ public class ClusterSearcher extends Searcher {
}
}
- private Result searchMultipleDocumentTypes(Searcher searcher, Query query, Execution execution) {
- Set<String> schemas = resolveSchemas(query, execution.context().getIndexFacts());
+ private Result searchMultipleDocumentTypes(Searcher searcher, Query query, Execution execution, Set<String> schemas) {
List<Query> queries = createQueries(query, schemas);
if (queries.size() == 1) {
return perSchemaSearch(searcher, queries.get(0), execution);
@@ -301,25 +293,7 @@ public class ClusterSearcher extends Searcher {
}
Set<String> resolveSchemas(Query query, IndexFacts indexFacts) {
- Set<String> restrict = query.getModel().getRestrict();
- if (restrict == null || restrict.isEmpty()) {
- Set<String> sources = query.getModel().getSources();
- return (sources == null || sources.isEmpty())
- ? schemas
- : new HashSet<>(indexFacts.newSession(sources, Collections.emptyList(), schemas).documentTypes());
- } else {
- return filterValidDocumentTypes(restrict);
- }
- }
-
- private Set<String> filterValidDocumentTypes(Collection<String> restrict) {
- Set<String> retval = new LinkedHashSet<>();
- for (String docType : restrict) {
- if (docType != null && schemas.contains(docType)) {
- retval.add(docType);
- }
- }
- return retval;
+ return schemaResolver.resolve(query, indexFacts);
}
private List<Query> createQueries(Query query, Set<String> docTypes) {
diff --git a/container-search/src/main/java/com/yahoo/prelude/cluster/SchemaResolver.java b/container-search/src/main/java/com/yahoo/prelude/cluster/SchemaResolver.java
new file mode 100644
index 00000000000..3a2125d1d38
--- /dev/null
+++ b/container-search/src/main/java/com/yahoo/prelude/cluster/SchemaResolver.java
@@ -0,0 +1,58 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+package com.yahoo.prelude.cluster;
+
+import com.yahoo.prelude.IndexFacts;
+import com.yahoo.prelude.fastsearch.DocumentdbInfoConfig;
+import com.yahoo.search.Query;
+import com.yahoo.search.searchchain.Execution;
+
+import java.util.Collection;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+/**
+ * Resolves schemas from query and execution context
+ *
+ * @author bjorncs
+ */
+class SchemaResolver {
+
+ private final Set<String> schemas;
+
+ SchemaResolver(DocumentdbInfoConfig cfg) {
+ this(cfg.documentdb().stream().map(DocumentdbInfoConfig.Documentdb::name).toList());
+ }
+
+ SchemaResolver(Collection<String> schemas) {
+ this.schemas = new LinkedHashSet<>(schemas);
+ }
+
+ Set<String> resolve(Query query, Execution execution) {
+ return resolve(query, execution.context().getIndexFacts());
+ }
+
+ Set<String> resolve(Query query, IndexFacts indexFacts) {
+ if (schemas.size() == 1) return Set.of(schemas.iterator().next());
+ var restrict = query.getModel().getRestrict();
+ if (restrict == null || restrict.isEmpty()) {
+ Set<String> sources = query.getModel().getSources();
+ return (sources == null || sources.isEmpty())
+ ? schemas
+ : new LinkedHashSet<>(indexFacts.newSession(sources, Set.of(), schemas).documentTypes());
+ } else {
+ return filterValidDocumentTypes(restrict);
+ }
+ }
+
+ private Set<String> filterValidDocumentTypes(Collection<String> restrict) {
+ Set<String> retval = new LinkedHashSet<>();
+ for (String docType : restrict) {
+ if (docType != null && schemas.contains(docType)) {
+ retval.add(docType);
+ }
+ }
+ return retval;
+ }
+
+}
diff --git a/container-search/src/main/java/com/yahoo/prelude/query/parser/CustomParser.java b/container-search/src/main/java/com/yahoo/prelude/query/parser/CustomParser.java
index e3b2278475b..2bd408220cd 100644
--- a/container-search/src/main/java/com/yahoo/prelude/query/parser/CustomParser.java
+++ b/container-search/src/main/java/com/yahoo/prelude/query/parser/CustomParser.java
@@ -6,7 +6,6 @@ import com.yahoo.prelude.IndexFacts;
import com.yahoo.prelude.query.Item;
import com.yahoo.search.query.parser.Parser;
-import java.util.Collections;
import java.util.Set;
/**
@@ -23,7 +22,7 @@ public interface CustomParser extends Parser {
Set<String> toSearch, IndexFacts indexFacts, String defaultIndexName) {
if (indexFacts == null)
indexFacts = new IndexFacts();
- return parse(queryToParse, filterToParse, parsingLanguage, indexFacts.newSession(toSearch, Collections.emptySet()), defaultIndexName);
+ return parse(queryToParse, filterToParse, parsingLanguage, indexFacts.newSession(toSearch, Set.of()), defaultIndexName);
}
Item parse(String queryToParse, String filterToParse, Language parsingLanguage,
diff --git a/container-search/src/main/java/com/yahoo/prelude/query/parser/Tokenizer.java b/container-search/src/main/java/com/yahoo/prelude/query/parser/Tokenizer.java
index c1d415b8e27..9952ec64d13 100644
--- a/container-search/src/main/java/com/yahoo/prelude/query/parser/Tokenizer.java
+++ b/container-search/src/main/java/com/yahoo/prelude/query/parser/Tokenizer.java
@@ -8,7 +8,6 @@ import com.yahoo.prelude.Index;
import com.yahoo.prelude.IndexFacts;
import com.yahoo.prelude.query.Substring;
-import java.util.Collections;
import java.util.List;
import static com.yahoo.prelude.query.parser.Token.Kind.*;
@@ -63,7 +62,7 @@ public final class Tokenizer {
* @return a read-only list of tokens. This list can only be used by this thread
*/
public List<Token> tokenize(String string) {
- return tokenize(string, new IndexFacts().newSession(Collections.emptySet(), Collections.emptySet()));
+ return tokenize(string, new IndexFacts().newSession());
}
/**
@@ -171,13 +170,10 @@ public final class Tokenizer {
// this is a heuristic to check whether we probably have reached the end of an URL element
for (int i = tokens.size() - 1; i >= 0; --i) {
switch (tokens.get(i).kind) {
- case COLON:
- if (i == indexLastExplicitlyChangedAt) return false;
- break;
- case SPACE:
- return true;
- default:
- // do nothing
+ case COLON -> { if (i == indexLastExplicitlyChangedAt) return false; }
+ case SPACE -> { return true; }
+ default -> { }
+ // do nothing
}
}
// really not sure whether we should choose false instead, on cause of the guard at
diff --git a/container-search/src/main/java/com/yahoo/prelude/querytransform/NonPhrasingSearcher.java b/container-search/src/main/java/com/yahoo/prelude/querytransform/NonPhrasingSearcher.java
index 6bae5175d84..1eb7eb8abe7 100644
--- a/container-search/src/main/java/com/yahoo/prelude/querytransform/NonPhrasingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/querytransform/NonPhrasingSearcher.java
@@ -22,7 +22,7 @@ import java.util.List;
@Before("transformedQuery")
public class NonPhrasingSearcher extends Searcher {
- private static final CompoundName suggestonly=new CompoundName("suggestonly");
+ private static final CompoundName suggestonly = CompoundName.from("suggestonly");
private PhraseMatcher phraseMatcher;
diff --git a/container-search/src/main/java/com/yahoo/prelude/querytransform/PhrasingSearcher.java b/container-search/src/main/java/com/yahoo/prelude/querytransform/PhrasingSearcher.java
index b67ddade7ee..4da61f5eeca 100644
--- a/container-search/src/main/java/com/yahoo/prelude/querytransform/PhrasingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/querytransform/PhrasingSearcher.java
@@ -29,7 +29,7 @@ import java.util.List;
@Provides(PhrasingSearcher.PHRASE_REPLACEMENT)
public class PhrasingSearcher extends Searcher {
- private static final CompoundName suggestonly = new CompoundName("suggestonly");
+ private static final CompoundName suggestonly = CompoundName.from("suggestonly");
public static final String PHRASE_REPLACEMENT = "PhraseReplacement";
diff --git a/container-search/src/main/java/com/yahoo/prelude/querytransform/RecallSearcher.java b/container-search/src/main/java/com/yahoo/prelude/querytransform/RecallSearcher.java
index 2c7d78c0671..9f8facd2b28 100644
--- a/container-search/src/main/java/com/yahoo/prelude/querytransform/RecallSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/querytransform/RecallSearcher.java
@@ -39,7 +39,7 @@ import static com.yahoo.prelude.querytransform.StemmingSearcher.STEMMING;
@Before({STEMMING, ACCENT_REMOVAL})
public class RecallSearcher extends Searcher {
- public static final CompoundName recallName=new CompoundName("recall");
+ public static final CompoundName recallName = CompoundName.from("recall");
@Override
public Result search(Query query, Execution execution) {
diff --git a/container-search/src/main/java/com/yahoo/prelude/querytransform/StemmingSearcher.java b/container-search/src/main/java/com/yahoo/prelude/querytransform/StemmingSearcher.java
index 6aa262ca49e..7c4bcb38c41 100644
--- a/container-search/src/main/java/com/yahoo/prelude/querytransform/StemmingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/querytransform/StemmingSearcher.java
@@ -9,11 +9,33 @@ import com.yahoo.language.Language;
import com.yahoo.language.Linguistics;
import com.yahoo.language.process.StemMode;
import com.yahoo.language.process.StemList;
+
+import java.util.ArrayList;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
import java.util.logging.Level;
import com.yahoo.prelude.Index;
import com.yahoo.prelude.IndexFacts;
-import com.yahoo.prelude.query.*;
+import com.yahoo.prelude.query.AndItem;
+import com.yahoo.prelude.query.AndSegmentItem;
+import com.yahoo.prelude.query.BlockItem;
+import com.yahoo.prelude.query.CompositeItem;
+import com.yahoo.prelude.query.Highlight;
+import com.yahoo.prelude.query.Item;
+import com.yahoo.prelude.query.PhraseItem;
+import com.yahoo.prelude.query.PhraseSegmentItem;
+import com.yahoo.prelude.query.PrefixItem;
+import com.yahoo.prelude.query.SegmentingRule;
+import com.yahoo.prelude.query.Substring;
+import com.yahoo.prelude.query.TaggableItem;
+import com.yahoo.prelude.query.TermItem;
+import com.yahoo.prelude.query.WordAlternativesItem;
import com.yahoo.prelude.query.WordAlternativesItem.Alternative;
+import com.yahoo.prelude.query.WordItem;
import com.yahoo.processing.request.CompoundName;
import com.yahoo.search.Query;
import com.yahoo.search.Result;
@@ -21,7 +43,6 @@ import com.yahoo.search.Searcher;
import com.yahoo.search.searchchain.Execution;
import com.yahoo.search.searchchain.PhaseNames;
-import java.util.*;
import static com.yahoo.prelude.querytransform.CJKSearcher.TERM_ORDER_RELAXATION;
@@ -46,7 +67,7 @@ public class StemmingSearcher extends Searcher {
}
public static final String STEMMING = "Stemming";
- public static final CompoundName DISABLE = new CompoundName("nostemming");
+ public static final CompoundName DISABLE = CompoundName.from("nostemming");
private final Linguistics linguistics;
public StemmingSearcher(Linguistics linguistics) {
@@ -107,15 +128,13 @@ public class StemmingSearcher extends Searcher {
}
private Map<Item, TaggableItem> populateReverseConnectivityMap(Item root, Map<Item, TaggableItem> reverseConnectivity) {
- if (root instanceof TaggableItem) {
- TaggableItem asTaggable = (TaggableItem) root;
+ if (root instanceof TaggableItem asTaggable) {
Item connectsTo = asTaggable.getConnectedItem();
if (connectsTo != null) {
reverseConnectivity.put(connectsTo, asTaggable);
}
}
- if (root instanceof CompositeItem && !(root instanceof BlockItem)) {
- CompositeItem c = (CompositeItem) root;
+ if (root instanceof CompositeItem c && !(root instanceof BlockItem)) {
for (Iterator<Item> i = c.getItemIterator(); i.hasNext();) {
Item item = i.next();
populateReverseConnectivityMap(item, reverseConnectivity);
@@ -134,8 +153,7 @@ public class StemmingSearcher extends Searcher {
}
if (item instanceof BlockItem) {
item = checkBlock((BlockItem) item, context);
- } else if (item instanceof CompositeItem) {
- CompositeItem comp = (CompositeItem) item;
+ } else if (item instanceof CompositeItem comp) {
ListIterator<Item> i = comp.getItemIterator();
while (i.hasNext()) {
@@ -220,8 +238,7 @@ public class StemmingSearcher extends Searcher {
copyAttributes(blockAsItem, composite);
composite.lock();
- if (composite instanceof PhraseSegmentItem) {
- PhraseSegmentItem replacement = (PhraseSegmentItem) composite;
+ if (composite instanceof PhraseSegmentItem replacement) {
setSignificance(replacement, current);
phraseSegmentConnectivity(current, context.reverseConnectivity, replacement);
}
@@ -258,10 +275,9 @@ public class StemmingSearcher extends Searcher {
}
private Connectivity getConnectivity(BlockItem current) {
- if (!(current instanceof TaggableItem)) {
+ if (!(current instanceof TaggableItem t)) {
return null;
}
- TaggableItem t = (TaggableItem) current;
if (t.getConnectedItem() == null) {
return null;
}
@@ -294,7 +310,7 @@ public class StemmingSearcher extends Searcher {
Substring substring,
boolean insidePhrase) {
String indexName = current.getIndexName();
- if (insidePhrase == false && ((index.getLiteralBoost() || index.getStemMode() == StemMode.ALL))) {
+ if (!insidePhrase && ((index.getLiteralBoost() || index.getStemMode() == StemMode.ALL))) {
List<Alternative> terms = new ArrayList<>(segment.size() + 1);
terms.add(new Alternative(current.stringValue(), 1.0d));
for (String term : segment) {
@@ -305,8 +321,7 @@ public class StemmingSearcher extends Searcher {
return alternatives;
}
}
- WordItem first = singleStemSegment((Item) current, segment.get(0), indexName, substring);
- return first;
+ return singleStemSegment((Item) current, segment.get(0), indexName, substring);
}
private void setMetaData(BlockItem current, Map<Item, TaggableItem> reverseConnectivity, TaggableItem replacement) {
@@ -353,14 +368,13 @@ public class StemmingSearcher extends Searcher {
if (current.getSegmentingRule() == SegmentingRule.LANGUAGE_DEFAULT)
return chooseComposite(current, parent, indexName);
- switch (current.getSegmentingRule()) { // TODO: Why for CJK only? The segmentingRule says nothing about being for CJK only
- case PHRASE: return createPhraseSegment(current, indexName);
- case BOOLEAN_AND: return createAndSegment(current);
- default:
- throw new IllegalArgumentException("Unknown segmenting rule: " + current.getSegmentingRule() +
- ". This is a bug in Vespa, as the implementation has gotten out of sync." +
- " Please create an issue.");
- }
+ return switch (current.getSegmentingRule()) { // TODO: Why for CJK only? The segmentingRule says nothing about being for CJK only
+ case PHRASE -> createPhraseSegment(current, indexName);
+ case BOOLEAN_AND -> createAndSegment(current);
+ default -> throw new IllegalArgumentException("Unknown segmenting rule: " + current.getSegmentingRule() +
+ ". This is a bug in Vespa, as the implementation has gotten out of sync." +
+ " Please create an issue.");
+ };
}
private AndSegmentItem createAndSegment(BlockItem current) {
diff --git a/container-search/src/main/java/com/yahoo/prelude/searcher/FieldCollapsingSearcher.java b/container-search/src/main/java/com/yahoo/prelude/searcher/FieldCollapsingSearcher.java
index ead6ad53715..9927880c476 100644
--- a/container-search/src/main/java/com/yahoo/prelude/searcher/FieldCollapsingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/searcher/FieldCollapsingSearcher.java
@@ -25,10 +25,10 @@ import java.util.Map;
@Before(PhaseNames.TRANSFORMED_QUERY)
public class FieldCollapsingSearcher extends Searcher {
- private static final CompoundName collapse = new CompoundName("collapse");
- private static final CompoundName collapsefield = new CompoundName("collapsefield");
- private static final CompoundName collapsesize = new CompoundName("collapsesize");
- private static final CompoundName collapseSummaryName = new CompoundName("collapse.summary");
+ private static final CompoundName collapse = CompoundName.from("collapse");
+ private static final CompoundName collapsefield = CompoundName.from("collapsefield");
+ private static final CompoundName collapsesize = CompoundName.from("collapsesize");
+ private static final CompoundName collapseSummaryName = CompoundName.from("collapse.summary");
/** Maximum number of queries to send next searcher */
private static final int maxQueries = 4;
diff --git a/container-search/src/main/java/com/yahoo/prelude/searcher/JSONDebugSearcher.java b/container-search/src/main/java/com/yahoo/prelude/searcher/JSONDebugSearcher.java
index 10b2cf0fdc7..7750bd17108 100644
--- a/container-search/src/main/java/com/yahoo/prelude/searcher/JSONDebugSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/searcher/JSONDebugSearcher.java
@@ -25,7 +25,7 @@ public class JSONDebugSearcher extends Searcher {
public static final String STRUCT_FIELD = "Structured data field (as json): ";
public static final String FEATURE_FIELD = "Feature data field (as json): ";
- private static CompoundName PROPERTYNAME = new CompoundName("dumpjson");
+ private static final CompoundName PROPERTYNAME = CompoundName.from("dumpjson");
@Override
public Result search(Query query, Execution execution) {
diff --git a/container-search/src/main/java/com/yahoo/prelude/searcher/MultipleResultsSearcher.java b/container-search/src/main/java/com/yahoo/prelude/searcher/MultipleResultsSearcher.java
index 3c61a361cbb..dd2c4a1da7f 100644
--- a/container-search/src/main/java/com/yahoo/prelude/searcher/MultipleResultsSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/searcher/MultipleResultsSearcher.java
@@ -10,7 +10,12 @@ import com.yahoo.search.result.Hit;
import com.yahoo.search.result.HitGroup;
import com.yahoo.search.searchchain.Execution;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
/**
* Groups hits according to document type.
@@ -21,9 +26,9 @@ import java.util.*;
public class MultipleResultsSearcher extends Searcher {
private final static String propertyPrefix = "multipleresultsets.";
- private static final CompoundName additionalHitsFactorName=new CompoundName(propertyPrefix + "additionalHitsFactor");
- private static final CompoundName maxTimesRetrieveHeterogeneousHitsName=new CompoundName(propertyPrefix + "maxTimesRetrieveHeterogeneousHits");
- private static final CompoundName numHits=new CompoundName(propertyPrefix + "numHits");
+ private static final CompoundName additionalHitsFactorName = CompoundName.from(propertyPrefix + "additionalHitsFactor");
+ private static final CompoundName maxTimesRetrieveHeterogeneousHitsName = CompoundName.from(propertyPrefix + "maxTimesRetrieveHeterogeneousHits");
+ private static final CompoundName numHits = CompoundName.from(propertyPrefix + "numHits");
@Override
public Result search(Query query, Execution e) {
@@ -162,9 +167,9 @@ public class MultipleResultsSearcher extends Searcher {
// Assumes that field sddocname is available
private static class PartitionedResult {
- private Map<String, HitGroup> resultSets = new HashMap<>();
+ private final Map<String, HitGroup> resultSets = new HashMap<>();
- private List<Hit> otherHits = new ArrayList<>();
+ private final List<Hit> otherHits = new ArrayList<>();
PartitionedResult(List<DocumentGroup> documentGroups,Result result) throws ParameterException {
for (DocumentGroup group : documentGroups)
@@ -174,9 +179,8 @@ public class MultipleResultsSearcher extends Searcher {
}
void addHits(Result result, boolean addOtherHits) {
- Iterator<Hit> i = result.hits().iterator();
- while (i.hasNext()) {
- add(i.next(), addOtherHits);
+ for (Hit hit : result.hits()) {
+ add(hit, addOtherHits);
}
}
diff --git a/container-search/src/main/java/com/yahoo/prelude/searcher/PosSearcher.java b/container-search/src/main/java/com/yahoo/prelude/searcher/PosSearcher.java
index 0df21df8842..fd4c9e89f3b 100644
--- a/container-search/src/main/java/com/yahoo/prelude/searcher/PosSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/searcher/PosSearcher.java
@@ -37,12 +37,11 @@ public class PosSearcher extends Searcher {
public static final String POSITION_PARSING = "PositionParsing";
- private static final CompoundName posBb = new CompoundName("pos.bb");
- private static final CompoundName posLl = new CompoundName("pos.ll");
- private static final CompoundName posXy = new CompoundName("pos.xy");
- private static final CompoundName posAttributeName = new CompoundName("pos.attribute");
- private static final CompoundName posRadius = new CompoundName("pos.radius");
- private static final CompoundName posUnits = new CompoundName("pos.units");
+ private static final CompoundName posBb = CompoundName.from("pos.bb");
+ private static final CompoundName posLl = CompoundName.from("pos.ll");
+ private static final CompoundName posXy = CompoundName.from("pos.xy");
+ private static final CompoundName posAttributeName = CompoundName.from("pos.attribute");
+ private static final CompoundName posRadius = CompoundName.from("pos.radius");
// according to wikipedia:
// Earth's equatorial radius = 6378137 meter - not used
@@ -117,13 +116,13 @@ public class PosSearcher extends Searcher {
if (radius == null) {
radiusdegrees = 50.0 * km2deg;
} else if (radius.endsWith("km")) {
- double radiuskm = Double.valueOf(radius.substring(0, radius.length()-2));
+ double radiuskm = Double.parseDouble(radius.substring(0, radius.length()-2));
radiusdegrees = radiuskm * km2deg;
} else if (radius.endsWith("m")) {
- double radiusm = Double.valueOf(radius.substring(0, radius.length()-1));
+ double radiusm = Double.parseDouble(radius.substring(0, radius.length()-1));
radiusdegrees = radiusm * km2deg / 1000.0;
} else if (radius.endsWith("mi")) {
- double radiusmiles = Double.valueOf(radius.substring(0, radius.length()-2));
+ double radiusmiles = Double.parseDouble(radius.substring(0, radius.length()-2));
radiusdegrees = radiusmiles * mi2deg;
} else {
radiusdegrees = Integer.parseInt(radius) * 0.000001;
@@ -151,15 +150,15 @@ public class PosSearcher extends Searcher {
double radiusdegrees = radiuskm * km2deg;
radiusUnits = (int)(radiusdegrees * 1000000);
} else if (radius.endsWith("km")) {
- double radiuskm = Double.valueOf(radius.substring(0, radius.length()-2));
+ double radiuskm = Double.parseDouble(radius.substring(0, radius.length()-2));
double radiusdegrees = radiuskm * km2deg;
radiusUnits = (int)(radiusdegrees * 1000000);
} else if (radius.endsWith("m")) {
- double radiusm = Double.valueOf(radius.substring(0, radius.length()-1));
+ double radiusm = Double.parseDouble(radius.substring(0, radius.length()-1));
double radiusdegrees = radiusm * km2deg / 1000.0;
radiusUnits = (int)(radiusdegrees * 1000000);
} else if (radius.endsWith("mi")) {
- double radiusmiles = Double.valueOf(radius.substring(0, radius.length()-2));
+ double radiusmiles = Double.parseDouble(radius.substring(0, radius.length()-2));
double radiusdegrees = radiusmiles * mi2deg;
radiusUnits = (int)(radiusdegrees * 1000000);
} else {
diff --git a/container-search/src/main/java/com/yahoo/prelude/searcher/ValidateSortingSearcher.java b/container-search/src/main/java/com/yahoo/prelude/searcher/ValidateSortingSearcher.java
index 5488334c838..701e5d57950 100644
--- a/container-search/src/main/java/com/yahoo/prelude/searcher/ValidateSortingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/searcher/ValidateSortingSearcher.java
@@ -3,16 +3,16 @@ package com.yahoo.prelude.searcher;
import com.yahoo.component.chain.dependencies.After;
import com.yahoo.component.chain.dependencies.Before;
-import com.yahoo.vespa.config.search.AttributesConfig;
-import com.yahoo.search.config.ClusterConfig;
import com.yahoo.container.QrSearchersConfig;
import com.yahoo.search.Query;
import com.yahoo.search.Result;
import com.yahoo.search.Searcher;
+import com.yahoo.search.config.ClusterConfig;
import com.yahoo.search.query.Sorting;
import com.yahoo.search.result.ErrorMessage;
import com.yahoo.search.searchchain.Execution;
import com.yahoo.search.searchchain.PhaseNames;
+import com.yahoo.vespa.config.search.AttributesConfig;
import java.util.HashMap;
import java.util.List;
diff --git a/container-search/src/main/java/com/yahoo/prelude/semantics/SemanticSearcher.java b/container-search/src/main/java/com/yahoo/prelude/semantics/SemanticSearcher.java
index 42a2b4f4e9b..5b3806485b5 100644
--- a/container-search/src/main/java/com/yahoo/prelude/semantics/SemanticSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/semantics/SemanticSearcher.java
@@ -29,9 +29,9 @@ import static com.yahoo.prelude.querytransform.StemmingSearcher.STEMMING;
@Before({PhaseNames.TRANSFORMED_QUERY, STEMMING})
public class SemanticSearcher extends Searcher {
- private static final CompoundName rulesRulebase = new CompoundName("rules.rulebase");
- private static final CompoundName rulesOff = new CompoundName("rules.off");
- private static final CompoundName tracelevelRules = new CompoundName("tracelevel.rules");
+ private static final CompoundName rulesRulebase = CompoundName.from("rules.rulebase");
+ private static final CompoundName rulesOff = CompoundName.from("rules.off");
+ private static final CompoundName tracelevelRules = CompoundName.from("tracelevel.rules");
/** The default rule base of this */
private RuleBase defaultRuleBase;
diff --git a/container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java b/container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java
index c13bdff1f50..de24196327e 100644
--- a/container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/statistics/StatisticsSearcher.java
@@ -56,7 +56,7 @@ import static com.yahoo.container.protect.Error.UNSPECIFIED;
@Before(PhaseNames.RAW_QUERY)
public class StatisticsSearcher extends Searcher {
- private static final CompoundName IGNORE_QUERY = new CompoundName("metrics.ignore");
+ private static final CompoundName IGNORE_QUERY = CompoundName.from("metrics.ignore");
private static final String MAX_QUERY_LATENCY_METRIC = ContainerMetrics.MAX_QUERY_LATENCY.baseName();
private static final String EMPTY_RESULTS_METRIC = ContainerMetrics.EMPTY_RESULTS.baseName();
private static final String HITS_PER_QUERY_METRIC = ContainerMetrics.HITS_PER_QUERY.baseName();
diff --git a/container-search/src/main/java/com/yahoo/search/Query.java b/container-search/src/main/java/com/yahoo/search/Query.java
index cff43e07d70..73ed89687fe 100644
--- a/container-search/src/main/java/com/yahoo/search/Query.java
+++ b/container-search/src/main/java/com/yahoo/search/Query.java
@@ -48,7 +48,6 @@ import com.yahoo.yolean.Exceptions;
import java.net.URI;
import java.nio.ByteBuffer;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -180,23 +179,23 @@ public class Query extends com.yahoo.processing.Request implements Cloneable {
//---------------- Static property handling ------------------------------------
- public static final CompoundName OFFSET = new CompoundName("offset");
- public static final CompoundName HITS = new CompoundName("hits");
+ public static final CompoundName OFFSET = CompoundName.from("offset");
+ public static final CompoundName HITS = CompoundName.from("hits");
- public static final CompoundName QUERY_PROFILE = new CompoundName("queryProfile");
- public static final CompoundName SEARCH_CHAIN = new CompoundName("searchChain");
+ public static final CompoundName QUERY_PROFILE = CompoundName.from("queryProfile");
+ public static final CompoundName SEARCH_CHAIN = CompoundName.from("searchChain");
- public static final CompoundName NO_CACHE = new CompoundName("noCache");
- public static final CompoundName GROUPING_SESSION_CACHE = new CompoundName("groupingSessionCache");
- public static final CompoundName TIMEOUT = new CompoundName("timeout");
+ public static final CompoundName NO_CACHE = CompoundName.from("noCache");
+ public static final CompoundName GROUPING_SESSION_CACHE = CompoundName.from("groupingSessionCache");
+ public static final CompoundName TIMEOUT = CompoundName.from("timeout");
/** @deprecated use Trace.LEVEL */
@Deprecated // TODO: Remove on Vespa 9
- public static final CompoundName TRACE_LEVEL = new CompoundName("traceLevel");
+ public static final CompoundName TRACE_LEVEL = CompoundName.from("traceLevel");
/** @deprecated use Trace.EXPLAIN_LEVEL */
@Deprecated // TODO: Remove on Vespa 9
- public static final CompoundName EXPLAIN_LEVEL = new CompoundName("explainLevel");
+ public static final CompoundName EXPLAIN_LEVEL = CompoundName.from("explainLevel");
private static final QueryProfileType argumentType;
static {
@@ -228,17 +227,17 @@ public class Query extends com.yahoo.processing.Request implements Cloneable {
private static final Map<String, CompoundName> propertyAliases;
static {
Map<String, CompoundName> propertyAliasesBuilder = new HashMap<>();
- addAliases(Query.getArgumentType(), CompoundName.empty, propertyAliasesBuilder);
+ addAliases(Query.getArgumentType(), "", propertyAliasesBuilder);
propertyAliases = ImmutableMap.copyOf(propertyAliasesBuilder);
}
- private static void addAliases(QueryProfileType arguments, CompoundName prefix, Map<String, CompoundName> aliases) {
+ private static void addAliases(QueryProfileType arguments, String prefix, Map<String, CompoundName> aliases) {
for (FieldDescription field : arguments.fields().values()) {
for (String alias : field.getAliases())
- aliases.put(alias, prefix.append(field.getName()));
+ aliases.put(alias, CompoundName.from(append(prefix, field.getName())));
if (field.getType() instanceof QueryProfileFieldType) {
var type = ((QueryProfileFieldType) field.getType()).getQueryProfileType();
if (type != null)
- addAliases(type, prefix.append(type.getComponentIdAsCompoundName()), aliases);
+ addAliases(type, append(prefix, type.getComponentIdAsCompoundName().toString()), aliases);
}
}
}
@@ -261,18 +260,18 @@ public class Query extends com.yahoo.processing.Request implements Cloneable {
/** Returns an unmodifiable list of all the native properties under a Query */
public static final List<CompoundName> nativeProperties =
- List.copyOf(namesUnder(CompoundName.empty, Query.getArgumentType()));
+ List.copyOf(namesUnder("", Query.getArgumentType()));
- private static List<CompoundName> namesUnder(CompoundName prefix, QueryProfileType type) {
- if (type == null) return Collections.emptyList(); // Names not known statically
+ private static List<CompoundName> namesUnder(String prefix, QueryProfileType type) {
+ if (type == null) return List.of(); // Names not known statically
List<CompoundName> names = new ArrayList<>();
for (Map.Entry<String, FieldDescription> field : type.fields().entrySet()) {
+ String name = append(prefix, field.getKey());
if (field.getValue().getType() instanceof QueryProfileFieldType) {
- names.addAll(namesUnder(prefix.append(field.getKey()),
- ((QueryProfileFieldType) field.getValue().getType()).getQueryProfileType()));
+ names.addAll(namesUnder(name, ((QueryProfileFieldType) field.getValue().getType()).getQueryProfileType()));
}
else {
- names.add(prefix.append(field.getKey()));
+ names.add(CompoundName.from(name));
}
}
return names;
@@ -426,28 +425,36 @@ public class Query extends com.yahoo.processing.Request implements Cloneable {
* dependent objects for the appropriate subset of the given property values
*/
private void setFieldsFrom(Properties properties, Map<String, String> context) {
- setFrom(CompoundName.empty, properties, Query.getArgumentType(), context);
+ setFrom("", properties, Query.getArgumentType(), context);
+ }
+
+ private static String append(String a, String b) {
+ if (a.isEmpty()) return b;
+ if (b.isEmpty()) return a;
+ return a + "." + b;
}
/**
* For each field in the given query profile type, take the corresponding value from originalProperties
* (if any) set it to properties(), recursively.
*/
- private void setFrom(CompoundName prefix, Properties originalProperties, QueryProfileType arguments, Map<String, String> context) {
- prefix = prefix.append(getPrefix(arguments));
+ private void setFrom(String prefix, Properties originalProperties, QueryProfileType arguments, Map<String, String> context) {
+ prefix = append(prefix, getPrefix(arguments).toString());
for (FieldDescription field : arguments.fields().values()) {
if (field.getType() == FieldType.genericQueryProfileType) { // Generic map
- CompoundName fullName = prefix.append(field.getCompoundName());
- for (Map.Entry<String, Object> entry : originalProperties.listProperties(fullName, context).entrySet()) {
- properties().set(fullName.append(entry.getKey()), entry.getValue(), context);
+ String fullName = append(prefix, field.getCompoundName().toString());
+ for (Map.Entry<String, Object> entry : originalProperties.listProperties(CompoundName.from(fullName), context).entrySet()) {
+ properties().set(CompoundName.from(append(fullName, entry.getKey())), entry.getValue(), context);
}
}
else if (field.getType() instanceof QueryProfileFieldType) { // Nested arguments
setFrom(prefix, originalProperties, ((QueryProfileFieldType)field.getType()).getQueryProfileType(), context);
}
else {
- CompoundName fullName = prefix.append(field.getCompoundName());
+ CompoundName fullName = prefix.isEmpty()
+ ? field.getCompoundName()
+ : CompoundName.from(append(prefix, field.getCompoundName().toString()));
Object value = originalProperties.get(fullName, context);
if (value != null) {
properties().set(fullName, value, context);
@@ -458,14 +465,15 @@ public class Query extends com.yahoo.processing.Request implements Cloneable {
/** Calls properties.set on all entries in requestMap */
private void setPropertiesFromRequestMap(Map<String, String> requestMap, Properties properties, boolean ignoreSelect) {
- for (var entry : requestMap.entrySet()) {
+ var entrySet = requestMap.entrySet();
+ for (var entry : entrySet) {
if (ignoreSelect && entry.getKey().equals(Select.SELECT)) continue;
if (RankFeatures.isFeatureName(entry.getKey())) continue; // Set these last
- properties.set(entry.getKey(), entry.getValue(), requestMap);
+ properties.set(CompoundName.from(entry.getKey()), entry.getValue(), requestMap);
}
- for (var entry : requestMap.entrySet()) {
+ for (var entry : entrySet) {
if ( ! RankFeatures.isFeatureName(entry.getKey())) continue;
- properties.set(entry.getKey(), entry.getValue(), requestMap);
+ properties.set(CompoundName.from(entry.getKey()), entry.getValue(), requestMap);
}
}
diff --git a/container-search/src/main/java/com/yahoo/search/dispatch/Dispatcher.java b/container-search/src/main/java/com/yahoo/search/dispatch/Dispatcher.java
index 09db1be4732..4e4b77422c1 100644
--- a/container-search/src/main/java/com/yahoo/search/dispatch/Dispatcher.java
+++ b/container-search/src/main/java/com/yahoo/search/dispatch/Dispatcher.java
@@ -52,7 +52,7 @@ public class Dispatcher extends AbstractComponent {
private static final int MAX_GROUP_SELECTION_ATTEMPTS = 3;
/** If set will control computation of how many hits will be fetched from each partition.*/
- public static final CompoundName topKProbability = CompoundName.fromComponents(DISPATCH, TOP_K_PROBABILITY);
+ public static final CompoundName topKProbability = CompoundName.from(DISPATCH + "." + TOP_K_PROBABILITY);
private final DispatchConfig dispatchConfig;
private final RpcResourcePool rpcResourcePool;
diff --git a/container-search/src/main/java/com/yahoo/search/dispatch/rpc/CompressService.java b/container-search/src/main/java/com/yahoo/search/dispatch/rpc/CompressService.java
index 9e7fc9b5b29..c10375a3014 100644
--- a/container-search/src/main/java/com/yahoo/search/dispatch/rpc/CompressService.java
+++ b/container-search/src/main/java/com/yahoo/search/dispatch/rpc/CompressService.java
@@ -13,7 +13,7 @@ import com.yahoo.search.Query;
*/
public class CompressService implements CompressPayload {
/** The compression method which will be used with rpc dispatch. "lz4" (default) and "none" is supported. */
- public final static CompoundName dispatchCompression = new CompoundName("dispatch.compression");
+ public static final CompoundName dispatchCompression = CompoundName.from("dispatch.compression");
private final Compressor compressor = new Compressor(CompressionType.LZ4, 5, 0.95, 256);
diff --git a/container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcInvokerFactory.java b/container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcInvokerFactory.java
index 4466b03a713..154002c4f77 100644
--- a/container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcInvokerFactory.java
+++ b/container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcInvokerFactory.java
@@ -20,11 +20,20 @@ public class RpcInvokerFactory extends InvokerFactory {
private final RpcConnectionPool rpcResourcePool;
private final CompressPayload compressor;
+ private final RpcProtobufFillInvoker.DecodePolicy decodeType;
+
+ private static RpcProtobufFillInvoker.DecodePolicy convert(DispatchConfig.SummaryDecodePolicy.Enum decoding) {
+ return switch (decoding) {
+ case EAGER -> RpcProtobufFillInvoker.DecodePolicy.EAGER;
+ case ONDEMAND -> RpcProtobufFillInvoker.DecodePolicy.ONDEMAND;
+ };
+ }
public RpcInvokerFactory(RpcConnectionPool rpcResourcePool, SearchGroups cluster, DispatchConfig dispatchConfig) {
super(cluster, dispatchConfig);
this.rpcResourcePool = rpcResourcePool;
this.compressor = new CompressService();
+ decodeType = convert(dispatchConfig.summaryDecodePolicy());
}
@Override
@@ -37,6 +46,7 @@ public class RpcInvokerFactory extends InvokerFactory {
Query query = result.getQuery();
boolean summaryNeedsQuery = searcher.summaryNeedsQuery(query);
- return new RpcProtobufFillInvoker(rpcResourcePool, compressor, searcher.getDocumentDatabase(query), searcher.getServerId(), summaryNeedsQuery);
+ return new RpcProtobufFillInvoker(rpcResourcePool, compressor, searcher.getDocumentDatabase(query),
+ searcher.getServerId(), decodeType, summaryNeedsQuery);
}
}
diff --git a/container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcProtobufFillInvoker.java b/container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcProtobufFillInvoker.java
index 4e538fb54dc..2bdafecfaba 100644
--- a/container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcProtobufFillInvoker.java
+++ b/container-search/src/main/java/com/yahoo/search/dispatch/rpc/RpcProtobufFillInvoker.java
@@ -21,6 +21,7 @@ import com.yahoo.search.result.ErrorMessage;
import com.yahoo.search.result.Hit;
import com.yahoo.slime.ArrayTraverser;
import com.yahoo.slime.BinaryFormat;
+import com.yahoo.slime.BinaryView;
import java.util.Iterator;
import java.util.List;
@@ -42,11 +43,14 @@ public class RpcProtobufFillInvoker extends FillInvoker {
private static final Logger log = Logger.getLogger(RpcProtobufFillInvoker.class.getName());
+ enum DecodePolicy {EAGER, ONDEMAND}
+
private final DocumentDatabase documentDb;
private final RpcConnectionPool resourcePool;
private final boolean summaryNeedsQuery;
private final String serverId;
private final CompressPayload compressor;
+ private final DecodePolicy decodePolicy;
private BlockingQueue<Pair<Client.ResponseOrError<ProtobufResponse>, List<FastHit>>> responses;
@@ -56,12 +60,14 @@ public class RpcProtobufFillInvoker extends FillInvoker {
/** The number of responses we should receive (and process) before this is complete */
private int outstandingResponses;
- RpcProtobufFillInvoker(RpcConnectionPool resourcePool, CompressPayload compressor, DocumentDatabase documentDb, String serverId, boolean summaryNeedsQuery) {
+ RpcProtobufFillInvoker(RpcConnectionPool resourcePool, CompressPayload compressor, DocumentDatabase documentDb,
+ String serverId, DecodePolicy decodePolicy, boolean summaryNeedsQuery) {
this.documentDb = documentDb;
this.resourcePool = resourcePool;
this.serverId = serverId;
this.summaryNeedsQuery = summaryNeedsQuery;
this.compressor = compressor;
+ this.decodePolicy = decodePolicy;
}
@Override
@@ -211,7 +217,9 @@ public class RpcProtobufFillInvoker extends FillInvoker {
private int fill(Result result, List<FastHit> hits, String summaryClass, byte[] payload) {
try {
var protobuf = SearchProtocol.DocsumReply.parseFrom(payload);
- var root = BinaryFormat.decode(protobuf.getSlimeSummaries().toByteArray()).get();
+ var root = (decodePolicy == DecodePolicy.ONDEMAND)
+ ? BinaryView.inspect(protobuf.getSlimeSummaries().toByteArray())
+ : BinaryFormat.decode(protobuf.getSlimeSummaries().toByteArray()).get();
var errors = root.field("errors");
boolean hasErrors = errors.valid() && (errors.entries() > 0);
if (hasErrors) {
diff --git a/container-search/src/main/java/com/yahoo/search/federation/FederationSearcher.java b/container-search/src/main/java/com/yahoo/search/federation/FederationSearcher.java
index 80a41ffdf22..b8be1dc210b 100644
--- a/container-search/src/main/java/com/yahoo/search/federation/FederationSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/federation/FederationSearcher.java
@@ -70,8 +70,8 @@ public class FederationSearcher extends ForkingSearcher {
private static final Logger log = Logger.getLogger(FederationSearcher.class.getName());
/** The name of the query property containing the source name added to the query to each source by this */
- public final static CompoundName SOURCENAME = new CompoundName("sourceName");
- public final static CompoundName PROVIDERNAME = new CompoundName("providerName");
+ public static final CompoundName SOURCENAME = CompoundName.from("sourceName");
+ public static final CompoundName PROVIDERNAME = CompoundName.from("providerName");
public static final String FEDERATION = "Federation";
public static final String LOG_COUNT_PREFIX = "count_";
@@ -686,33 +686,6 @@ public class FederationSearcher extends ForkingSearcher {
}
- private static class CompoundKey {
-
- private final String sourceName;
- private final String propertyName;
-
- CompoundKey(String sourceName, String propertyName) {
- this.sourceName = sourceName;
- this.propertyName = propertyName;
- }
-
- @Override
- public int hashCode() {
- return sourceName.hashCode() ^ propertyName.hashCode();
- }
-
- @Override
- public boolean equals(Object o) {
- CompoundKey rhs = (CompoundKey) o;
- return sourceName.equals(rhs.sourceName) && propertyName.equals(rhs.propertyName);
- }
-
- @Override
- public String toString() {
- return sourceName + '.' + propertyName;
- }
- }
-
private static class Window {
private final int hits;
diff --git a/container-search/src/main/java/com/yahoo/search/grouping/GroupingQueryParser.java b/container-search/src/main/java/com/yahoo/search/grouping/GroupingQueryParser.java
index 54194221958..180687d5274 100644
--- a/container-search/src/main/java/com/yahoo/search/grouping/GroupingQueryParser.java
+++ b/container-search/src/main/java/com/yahoo/search/grouping/GroupingQueryParser.java
@@ -38,13 +38,13 @@ import java.util.TimeZone;
public class GroupingQueryParser extends Searcher {
public static final String SELECT_PARAMETER_PARSING = "SelectParameterParsing";
- public static final CompoundName PARAM_CONTINUE = new CompoundName("continue");
- public static final CompoundName PARAM_REQUEST = new CompoundName(Select.SELECT);
- public static final CompoundName PARAM_TIMEZONE = new CompoundName("timezone");
- @Beta public static final CompoundName PARAM_DEFAULT_MAX_HITS = new CompoundName("grouping.defaultMaxHits");
- @Beta public static final CompoundName PARAM_DEFAULT_MAX_GROUPS = new CompoundName("grouping.defaultMaxGroups");
- @Beta public static final CompoundName PARAM_DEFAULT_PRECISION_FACTOR = new CompoundName("grouping.defaultPrecisionFactor");
- @Beta public static final CompoundName GROUPING_GLOBAL_MAX_GROUPS = new CompoundName("grouping.globalMaxGroups");
+ public static final CompoundName PARAM_CONTINUE = CompoundName.from("continue");
+ public static final CompoundName PARAM_REQUEST = CompoundName.from(Select.SELECT);
+ public static final CompoundName PARAM_TIMEZONE = CompoundName.from("timezone");
+ @Beta public static final CompoundName PARAM_DEFAULT_MAX_HITS = CompoundName.from("grouping.defaultMaxHits");
+ @Beta public static final CompoundName PARAM_DEFAULT_MAX_GROUPS = CompoundName.from("grouping.defaultMaxGroups");
+ @Beta public static final CompoundName PARAM_DEFAULT_PRECISION_FACTOR = CompoundName.from("grouping.defaultPrecisionFactor");
+ @Beta public static final CompoundName GROUPING_GLOBAL_MAX_GROUPS = CompoundName.from("grouping.globalMaxGroups");
private static final ThreadLocal<ZoneCache> zoneCache = new ThreadLocal<>();
@Override
@@ -121,7 +121,6 @@ public class GroupingQueryParser extends Searcher {
return val != null ? OptionalDouble.of(val) : OptionalDouble.empty();
}
- @SuppressWarnings("serial")
private static class ZoneCache extends LinkedHashMap<String, TimeZone> {
ZoneCache() {
diff --git a/container-search/src/main/java/com/yahoo/search/grouping/GroupingValidator.java b/container-search/src/main/java/com/yahoo/search/grouping/GroupingValidator.java
index 35cf72d6124..878fbbb733d 100644
--- a/container-search/src/main/java/com/yahoo/search/grouping/GroupingValidator.java
+++ b/container-search/src/main/java/com/yahoo/search/grouping/GroupingValidator.java
@@ -37,7 +37,7 @@ import static com.yahoo.search.grouping.GroupingQueryParser.SELECT_PARAMETER_PAR
public class GroupingValidator extends Searcher {
public static final String GROUPING_VALIDATED = "GroupingValidated";
- public static final CompoundName PARAM_ENABLED = new CompoundName("validate_" + GroupingQueryParser.PARAM_REQUEST);
+ public static final CompoundName PARAM_ENABLED = CompoundName.from("validate_" + GroupingQueryParser.PARAM_REQUEST);
private final HashMap<String, AttributesConfig.Attribute> attributes = new HashMap<>();
private final String clusterName;
private final boolean enabled;
diff --git a/container-search/src/main/java/com/yahoo/search/grouping/UniqueGroupingSearcher.java b/container-search/src/main/java/com/yahoo/search/grouping/UniqueGroupingSearcher.java
index a1446367f79..27d8a76511d 100644
--- a/container-search/src/main/java/com/yahoo/search/grouping/UniqueGroupingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/grouping/UniqueGroupingSearcher.java
@@ -42,7 +42,7 @@ import java.util.logging.Logger;
@Before(PhaseNames.TRANSFORMED_QUERY)
public class UniqueGroupingSearcher extends Searcher {
- public static final CompoundName PARAM_UNIQUE = new CompoundName("unique");
+ public static final CompoundName PARAM_UNIQUE = CompoundName.from("unique");
private static final Logger log = Logger.getLogger(UniqueGroupingSearcher.class.getName());
private static final HitOrderer NOP_ORDERER = new HitOrderer() {
@@ -144,17 +144,13 @@ public class UniqueGroupingSearcher extends Searcher {
for (Sorting.FieldOrder fieldOrder : sortingSpec.fieldOrders()) {
Sorting.Order sortOrder = fieldOrder.getSortOrder();
switch (sortOrder) {
- case ASCENDING:
- case UNDEFINED:
- // When we want ascending order, the hit with the smallest value should come first (and be surfaced).
- orderingClause.add(new MinAggregator(new AttributeValue(fieldOrder.getFieldName())));
- break;
- case DESCENDING:
- // When we sort in descending order, the hit with the largest value should come first (and be surfaced).
- orderingClause.add(new NegFunction(new MaxAggregator(new AttributeValue(fieldOrder.getFieldName()))));
- break;
- default:
- throw new UnsupportedOperationException("Can not handle sort order " + sortOrder + ".");
+ case ASCENDING, UNDEFINED ->
+ // When we want ascending order, the hit with the smallest value should come first (and be surfaced).
+ orderingClause.add(new MinAggregator(new AttributeValue(fieldOrder.getFieldName())));
+ case DESCENDING ->
+ // When we sort in descending order, the hit with the largest value should come first (and be surfaced).
+ orderingClause.add(new NegFunction(new MaxAggregator(new AttributeValue(fieldOrder.getFieldName()))));
+ default -> throw new UnsupportedOperationException("Can not handle sort order " + sortOrder + ".");
}
}
return orderingClause;
@@ -170,18 +166,13 @@ public class UniqueGroupingSearcher extends Searcher {
GroupingExpression groupingClause = null;
for (Sorting.FieldOrder fieldOrder : sortingSpec.fieldOrders()) {
Sorting.Order sortOrder = fieldOrder.getSortOrder();
- switch (sortOrder) {
- case ASCENDING:
- case UNDEFINED:
- groupingClause = new AttributeValue(fieldOrder.getFieldName());
- break;
- case DESCENDING:
- // To sort descending, just take the negative. This is the most common case
- groupingClause = new NegFunction(new AttributeValue(fieldOrder.getFieldName()));
- break;
- default:
- throw new UnsupportedOperationException("Can not handle sort order " + sortOrder + ".");
- }
+ groupingClause = switch (sortOrder) {
+ case ASCENDING, UNDEFINED -> new AttributeValue(fieldOrder.getFieldName());
+ case DESCENDING ->
+ // To sort descending, just take the negative. This is the most common case
+ new NegFunction(new AttributeValue(fieldOrder.getFieldName()));
+ default -> throw new UnsupportedOperationException("Can not handle sort order " + sortOrder + ".");
+ };
}
return groupingClause;
}
diff --git a/container-search/src/main/java/com/yahoo/search/grouping/result/FlatteningSearcher.java b/container-search/src/main/java/com/yahoo/search/grouping/result/FlatteningSearcher.java
index e56b88c3d3e..027ba92f587 100644
--- a/container-search/src/main/java/com/yahoo/search/grouping/result/FlatteningSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/grouping/result/FlatteningSearcher.java
@@ -22,7 +22,7 @@ import java.util.Iterator;
@Before(GroupingExecutor.COMPONENT_NAME)
public class FlatteningSearcher extends Searcher {
- private final CompoundName flatten = CompoundName.fromComponents("grouping", "flatten");
+ private final CompoundName flatten = CompoundName.from("grouping.flatten");
@Override
public Result search(Query query, Execution execution) {
diff --git a/container-search/src/main/java/com/yahoo/search/grouping/vespa/GroupingExecutor.java b/container-search/src/main/java/com/yahoo/search/grouping/vespa/GroupingExecutor.java
index 2ba33f60ea1..c85dbed83c8 100644
--- a/container-search/src/main/java/com/yahoo/search/grouping/vespa/GroupingExecutor.java
+++ b/container-search/src/main/java/com/yahoo/search/grouping/vespa/GroupingExecutor.java
@@ -379,7 +379,7 @@ public class GroupingExecutor extends Searcher {
}
private static CompoundName newCompoundName(String name) {
- return new CompoundName(GroupingExecutor.class.getName() + "." + name);
+ return CompoundName.from(GroupingExecutor.class.getName() + "." + name);
}
private static class RequestContext {
diff --git a/container-search/src/main/java/com/yahoo/search/handler/SearchHandler.java b/container-search/src/main/java/com/yahoo/search/handler/SearchHandler.java
index fac6c598e4b..3b4d15f788f 100644
--- a/container-search/src/main/java/com/yahoo/search/handler/SearchHandler.java
+++ b/container-search/src/main/java/com/yahoo/search/handler/SearchHandler.java
@@ -75,8 +75,8 @@ public class SearchHandler extends LoggingRequestHandler {
// max number of threads for the executor for this handler
private final int maxThreads;
- private static final CompoundName DETAILED_TIMING_LOGGING = new CompoundName("trace.timingDetails");
- private static final CompoundName FORCE_TIMESTAMPS = new CompoundName("trace.timestamps");
+ private static final CompoundName DETAILED_TIMING_LOGGING = CompoundName.from("trace.timingDetails");
+ private static final CompoundName FORCE_TIMESTAMPS = CompoundName.from("trace.timestamps");
/** Event name for number of connections to the search subsystem */
private static final String SEARCH_CONNECTIONS = "search_connections";
diff --git a/container-search/src/main/java/com/yahoo/search/intent/model/IntentModel.java b/container-search/src/main/java/com/yahoo/search/intent/model/IntentModel.java
index 9aafd2c4693..0c4b22fbbf2 100644
--- a/container-search/src/main/java/com/yahoo/search/intent/model/IntentModel.java
+++ b/container-search/src/main/java/com/yahoo/search/intent/model/IntentModel.java
@@ -5,7 +5,12 @@ import com.yahoo.search.Query;
import com.yahoo.processing.request.CompoundName;
import com.yahoo.text.interpretation.Interpretation;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
/**
* This is the root node of an intent model.
@@ -19,9 +24,9 @@ import java.util.*;
public class IntentModel extends ParentNode<InterpretationNode> {
/** The name of the property carrying the intent model string: intentModel */
- public static final CompoundName intentModelStringName=new CompoundName("intentModel");
+ public static final CompoundName intentModelStringName = CompoundName.from("intentModel");
/** The name of the property carrying the intent model object: IntentModel */
- public static final CompoundName intentModelObjectName=new CompoundName("IntentModel");
+ public static final CompoundName intentModelObjectName = CompoundName.from("IntentModel");
private static final InterpretationNodeComparator inodeComp = new InterpretationNodeComparator();
@@ -45,7 +50,7 @@ public class IntentModel extends ParentNode<InterpretationNode> {
/** Sort interpretations by descending score order */
public void sortChildren() {
- Collections.sort(children(), inodeComp);
+ children().sort(inodeComp);
}
/**
diff --git a/container-search/src/main/java/com/yahoo/search/pagetemplates/PageTemplateSearcher.java b/container-search/src/main/java/com/yahoo/search/pagetemplates/PageTemplateSearcher.java
index c23627accf4..5fef1821de2 100644
--- a/container-search/src/main/java/com/yahoo/search/pagetemplates/PageTemplateSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/pagetemplates/PageTemplateSearcher.java
@@ -72,13 +72,13 @@ import java.util.Map;
public class PageTemplateSearcher extends Searcher {
/** The name of the query property containing the resolved candidate page template list */
- public static final CompoundName pagePageTemplateListName=new CompoundName("page.PageTemplateList");
+ public static final CompoundName pagePageTemplateListName = CompoundName.from("page.PageTemplateList");
/** The name of the query property containing a list of candidate pages to consider */
- public static final CompoundName pageIdListName=new CompoundName("page.idList");
+ public static final CompoundName pageIdListName = CompoundName.from("page.idList");
/** The name of the query property containing the page id to use */
- public static final CompoundName pageIdName=new CompoundName("page.id");
+ public static final CompoundName pageIdName = CompoundName.from("page.id");
/** The name of the query property containing the resolver id to use */
- public static final CompoundName pageResolverName=new CompoundName("page.resolver");
+ public static final CompoundName pageResolverName = CompoundName.from("page.resolver");
private final ResolverRegistry resolverRegistry;
diff --git a/container-search/src/main/java/com/yahoo/search/query/Model.java b/container-search/src/main/java/com/yahoo/search/query/Model.java
index 190ad675015..09b2f394f20 100644
--- a/container-search/src/main/java/com/yahoo/search/query/Model.java
+++ b/container-search/src/main/java/com/yahoo/search/query/Model.java
@@ -71,13 +71,13 @@ public class Model implements Cloneable {
argumentType.addField(new FieldDescription(SEARCH_PATH, "string", "searchpath"));
argumentType.addField(new FieldDescription(RESTRICT, "string", "restrict"));
argumentType.freeze();
- argumentTypeName = new CompoundName(argumentType.getId().getName());
+ argumentTypeName = CompoundName.from(argumentType.getId().getName());
}
public static QueryProfileType getArgumentType() { return argumentType; }
/** The name of the query property used for generating hit count estimate queries. */
- public static final CompoundName ESTIMATE = new CompoundName("hitcountestimate"); // TODO: Cleanup
+ public static final CompoundName ESTIMATE = CompoundName.from("hitcountestimate"); // TODO: Cleanup
private String encoding = null;
private String queryString = "";
diff --git a/container-search/src/main/java/com/yahoo/search/query/Ranking.java b/container-search/src/main/java/com/yahoo/search/query/Ranking.java
index e8738a19412..5426268d173 100644
--- a/container-search/src/main/java/com/yahoo/search/query/Ranking.java
+++ b/container-search/src/main/java/com/yahoo/search/query/Ranking.java
@@ -27,7 +27,7 @@ import java.util.Objects;
public class Ranking implements Cloneable {
/** An alias for listing features */
- public static final CompoundName RANKFEATURES = new CompoundName("rankfeatures");
+ public static final CompoundName RANKFEATURES = CompoundName.from("rankfeatures");
/** The type representing the property arguments consumed by this */
private static final QueryProfileType argumentType;
@@ -70,7 +70,7 @@ public class Ranking implements Cloneable {
argumentType.addField(new FieldDescription(FEATURES, "query-profile", "rankfeature input")); // Repeated at the end of RankFeatures
argumentType.addField(new FieldDescription(PROPERTIES, "query-profile", "rankproperty"));
argumentType.freeze();
- argumentTypeName = new CompoundName(argumentType.getId().getName());
+ argumentTypeName = CompoundName.from(argumentType.getId().getName());
}
public static QueryProfileType getArgumentType() { return argumentType; }
diff --git a/container-search/src/main/java/com/yahoo/search/query/profile/compiled/CompiledQueryProfile.java b/container-search/src/main/java/com/yahoo/search/query/profile/compiled/CompiledQueryProfile.java
index ad9d3f4c1a5..ae531c67dd1 100644
--- a/container-search/src/main/java/com/yahoo/search/query/profile/compiled/CompiledQueryProfile.java
+++ b/container-search/src/main/java/com/yahoo/search/query/profile/compiled/CompiledQueryProfile.java
@@ -169,7 +169,7 @@ public class CompiledQueryProfile extends AbstractComponent implements Cloneable
}
public final Object get(String name) {
- return get(name, Collections.emptyMap());
+ return get(name, Map.of());
}
public final Object get(String name, Map<String, String> context) {
return get(name, context, new QueryProfileProperties(this));
diff --git a/container-search/src/main/java/com/yahoo/search/query/profile/types/FieldDescription.java b/container-search/src/main/java/com/yahoo/search/query/profile/types/FieldDescription.java
index 8fdbf8b2281..719a5a2c281 100644
--- a/container-search/src/main/java/com/yahoo/search/query/profile/types/FieldDescription.java
+++ b/container-search/src/main/java/com/yahoo/search/query/profile/types/FieldDescription.java
@@ -64,7 +64,7 @@ public class FieldDescription implements Comparable<FieldDescription> {
}
public FieldDescription(String name, FieldType type, String aliases, boolean mandatory, boolean overridable) {
- this(new CompoundName(name), type, aliases, mandatory, overridable);
+ this(CompoundName.from(name), type, aliases, mandatory, overridable);
}
/**
diff --git a/container-search/src/main/java/com/yahoo/search/query/profile/types/QueryProfileType.java b/container-search/src/main/java/com/yahoo/search/query/profile/types/QueryProfileType.java
index 3da2ad53f9a..9cbfe5b7112 100644
--- a/container-search/src/main/java/com/yahoo/search/query/profile/types/QueryProfileType.java
+++ b/container-search/src/main/java/com/yahoo/search/query/profile/types/QueryProfileType.java
@@ -52,7 +52,7 @@ public class QueryProfileType extends FreezableSimpleComponent {
private QueryProfileType(ComponentId id, Map<String, FieldDescription> fields, List<QueryProfileType> inherited) {
super(id);
QueryProfile.validateName(id.getName());
- componentIdAsCompoundName = new CompoundName(getId().getName());
+ componentIdAsCompoundName = CompoundName.from(getId().getName());
this.fields = fields;
this.inherited = inherited;
}
@@ -318,10 +318,9 @@ public class QueryProfileType extends FreezableSimpleComponent {
QueryProfileType type = null;
FieldDescription fieldDescription = getField(name);
if (fieldDescription != null) {
- if ( ! (fieldDescription.getType() instanceof QueryProfileFieldType))
+ if ( ! (fieldDescription.getType() instanceof QueryProfileFieldType fieldType))
throw new IllegalArgumentException("Cannot use name '" + name + "' as a prefix because it is " +
"already a " + fieldDescription.getType());
- QueryProfileFieldType fieldType = (QueryProfileFieldType) fieldDescription.getType();
type = fieldType.getQueryProfileType();
}
@@ -399,8 +398,7 @@ public class QueryProfileType extends FreezableSimpleComponent {
@Override
public boolean equals(Object o) {
if (o == this) return true;
- if ( ! (o instanceof QueryProfileType)) return false;
- QueryProfileType other = (QueryProfileType)o;
+ if ( ! (o instanceof QueryProfileType other)) return false;
return other.getId().equals(this.getId());
}
diff --git a/container-search/src/main/java/com/yahoo/search/query/properties/DefaultProperties.java b/container-search/src/main/java/com/yahoo/search/query/properties/DefaultProperties.java
index 221368afeb6..6ccf4792efb 100644
--- a/container-search/src/main/java/com/yahoo/search/query/properties/DefaultProperties.java
+++ b/container-search/src/main/java/com/yahoo/search/query/properties/DefaultProperties.java
@@ -16,9 +16,9 @@ import java.util.Map;
*/
public final class DefaultProperties extends Properties {
- public static final CompoundName MAX_OFFSET = new CompoundName("maxOffset");
- public static final CompoundName MAX_HITS = new CompoundName("maxHits");
- public static final CompoundName MAX_QUERY_ITEMS = new CompoundName("maxQueryItems");
+ public static final CompoundName MAX_OFFSET = CompoundName.from("maxOffset");
+ public static final CompoundName MAX_HITS = CompoundName.from("maxHits");
+ public static final CompoundName MAX_QUERY_ITEMS = CompoundName.from("maxQueryItems");
public static final QueryProfileType argumentType = new QueryProfileType("DefaultProperties");
diff --git a/container-search/src/main/java/com/yahoo/search/query/properties/SubProperties.java b/container-search/src/main/java/com/yahoo/search/query/properties/SubProperties.java
index 42fb4e44040..4fc9dfa03cd 100644
--- a/container-search/src/main/java/com/yahoo/search/query/properties/SubProperties.java
+++ b/container-search/src/main/java/com/yahoo/search/query/properties/SubProperties.java
@@ -10,14 +10,16 @@ import java.util.Map;
* A wrapper around a chain of property objects that prefixes all gets/sets with a given path
*
* @author Arne Bergene Fossaa
+ * @deprecated Unused and will go away on vespa 9
*/
+@Deprecated (forRemoval = true)
public class SubProperties extends com.yahoo.search.query.Properties {
final private CompoundName pathPrefix;
final private Properties parent;
public SubProperties(String pathPrefix, Properties properties) {
- this(new CompoundName(pathPrefix),properties);
+ this(CompoundName.from(pathPrefix), properties);
}
public SubProperties(CompoundName pathPrefix, Properties properties) {
diff --git a/container-search/src/main/java/com/yahoo/search/query/ranking/SoftTimeout.java b/container-search/src/main/java/com/yahoo/search/query/ranking/SoftTimeout.java
index 30ee2f473b7..c49b14f0978 100644
--- a/container-search/src/main/java/com/yahoo/search/query/ranking/SoftTimeout.java
+++ b/container-search/src/main/java/com/yahoo/search/query/ranking/SoftTimeout.java
@@ -24,8 +24,7 @@ public class SoftTimeout implements Cloneable {
public static final String TAILCOST = "tailcost";
/** The full property name for turning softtimeout on or off */
- public static final CompoundName enableProperty =
- CompoundName.fromComponents(Ranking.RANKING, Ranking.SOFTTIMEOUT, ENABLE);
+ public static final CompoundName enableProperty = CompoundName.from(Ranking.RANKING + "." + Ranking.SOFTTIMEOUT + "." + ENABLE);
static {
argumentType = new QueryProfileType(Ranking.SOFTTIMEOUT);
diff --git a/container-search/src/main/java/com/yahoo/search/query/rewrite/RewriterConstants.java b/container-search/src/main/java/com/yahoo/search/query/rewrite/RewriterConstants.java
index b0fce45de46..eeb8b675ae1 100644
--- a/container-search/src/main/java/com/yahoo/search/query/rewrite/RewriterConstants.java
+++ b/container-search/src/main/java/com/yahoo/search/query/rewrite/RewriterConstants.java
@@ -42,7 +42,7 @@ public class RewriterConstants {
public static final String REWRITER_CHAIN = "QRWChain";
/** Name for rewrite metadata retrieval from query properties */
- public static final CompoundName REWRITE_META = new CompoundName("RewriteMeta");
+ public static final CompoundName REWRITE_META = CompoundName.from("RewriteMeta");
/** Name for rewritten field retrieval from query properties */
public static final String REWRITTEN = "Rewritten";
diff --git a/container-search/src/main/java/com/yahoo/search/querytransform/BooleanSearcher.java b/container-search/src/main/java/com/yahoo/search/querytransform/BooleanSearcher.java
index f43be20e0ac..91315fe1bb3 100644
--- a/container-search/src/main/java/com/yahoo/search/querytransform/BooleanSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/querytransform/BooleanSearcher.java
@@ -28,9 +28,9 @@ import static com.yahoo.yolean.Exceptions.toMessageString;
@Provides(BooleanSearcher.PREDICATE)
public class BooleanSearcher extends Searcher {
- private static final CompoundName FIELD = new CompoundName("boolean.field");
- private static final CompoundName ATTRIBUTES = new CompoundName("boolean.attributes");
- private static final CompoundName RANGE_ATTRIBUTES = new CompoundName("boolean.rangeAttributes");
+ private static final CompoundName FIELD = CompoundName.from("boolean.field");
+ private static final CompoundName ATTRIBUTES = CompoundName.from("boolean.attributes");
+ private static final CompoundName RANGE_ATTRIBUTES = CompoundName.from("boolean.rangeAttributes");
public static final String PREDICATE = "predicate";
@Override
diff --git a/container-search/src/main/java/com/yahoo/search/querytransform/SortingDegrader.java b/container-search/src/main/java/com/yahoo/search/querytransform/SortingDegrader.java
index 75b06c29193..81881685a1d 100644
--- a/container-search/src/main/java/com/yahoo/search/querytransform/SortingDegrader.java
+++ b/container-search/src/main/java/com/yahoo/search/querytransform/SortingDegrader.java
@@ -41,9 +41,9 @@ public class SortingDegrader extends Searcher {
/** Set this to false in query.properties to turn off degrading. Default: on */
// (this is not called ranking.sorting.degrading because it should not be part of the query object model
- public static final CompoundName DEGRADING = new CompoundName("sorting.degrading");
+ public static final CompoundName DEGRADING = CompoundName.from("sorting.degrading");
- public static final CompoundName PAGINATION = new CompoundName("to_be_removed_pagination");
+ public static final CompoundName PAGINATION = CompoundName.from("to_be_removed_pagination");
@Override
public Result search(Query query, Execution execution) {
diff --git a/container-search/src/main/java/com/yahoo/search/querytransform/WandSearcher.java b/container-search/src/main/java/com/yahoo/search/querytransform/WandSearcher.java
index 89245eaf137..bfac668b6c4 100644
--- a/container-search/src/main/java/com/yahoo/search/querytransform/WandSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/querytransform/WandSearcher.java
@@ -63,12 +63,12 @@ public class WandSearcher extends Searcher {
*/
private static class InputResolver {
- private static final CompoundName WAND_FIELD = new CompoundName("wand.field");
- private static final CompoundName WAND_TOKENS = new CompoundName("wand.tokens");
- private static final CompoundName WAND_HEAP_SIZE = new CompoundName("wand.heapSize");
- private static final CompoundName WAND_TYPE = new CompoundName("wand.type");
- private static final CompoundName WAND_SCORE_THRESHOLD = new CompoundName("wand.scoreThreshold");
- private static final CompoundName WAND_THRESHOLD_BOOST_FACTOR = new CompoundName("wand.thresholdBoostFactor");
+ private static final CompoundName WAND_FIELD = CompoundName.from("wand.field");
+ private static final CompoundName WAND_TOKENS = CompoundName.from("wand.tokens");
+ private static final CompoundName WAND_HEAP_SIZE = CompoundName.from("wand.heapSize");
+ private static final CompoundName WAND_TYPE = CompoundName.from("wand.type");
+ private static final CompoundName WAND_SCORE_THRESHOLD = CompoundName.from("wand.scoreThreshold");
+ private static final CompoundName WAND_THRESHOLD_BOOST_FACTOR = CompoundName.from("wand.thresholdBoostFactor");
private final String fieldName;
private final WandType wandType;
private final Map<Object, Integer> tokens;
diff --git a/container-search/src/main/java/com/yahoo/search/querytransform/WeakAndReplacementSearcher.java b/container-search/src/main/java/com/yahoo/search/querytransform/WeakAndReplacementSearcher.java
index 2d6e059342e..9aa7a9d998d 100644
--- a/container-search/src/main/java/com/yahoo/search/querytransform/WeakAndReplacementSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/querytransform/WeakAndReplacementSearcher.java
@@ -21,8 +21,8 @@ import com.yahoo.yolean.chain.After;
*/
@After(MinimalQueryInserter.EXTERNAL_YQL)
public class WeakAndReplacementSearcher extends Searcher {
- static final CompoundName WEAKAND_REPLACE = new CompoundName("weakAnd.replace");
- static final CompoundName WAND_HITS = new CompoundName("wand.hits");
+ static final CompoundName WEAKAND_REPLACE = CompoundName.from("weakAnd.replace");
+ static final CompoundName WAND_HITS = CompoundName.from("wand.hits");
@Override public Result search(Query query, Execution execution) {
if (!query.properties().getBoolean(WEAKAND_REPLACE)) {
diff --git a/container-search/src/main/java/com/yahoo/search/ranking/GlobalPhaseRanker.java b/container-search/src/main/java/com/yahoo/search/ranking/GlobalPhaseRanker.java
index 2c6ab9e9367..dd419d69315 100644
--- a/container-search/src/main/java/com/yahoo/search/ranking/GlobalPhaseRanker.java
+++ b/container-search/src/main/java/com/yahoo/search/ranking/GlobalPhaseRanker.java
@@ -4,11 +4,14 @@ package com.yahoo.search.ranking;
import com.yahoo.component.annotation.Inject;
import com.yahoo.search.Query;
import com.yahoo.search.Result;
+import com.yahoo.search.query.Sorting;
import com.yahoo.search.ranking.RankProfilesEvaluator.GlobalPhaseData;
+import com.yahoo.search.result.ErrorMessage;
import com.yahoo.tensor.Tensor;
import java.util.ArrayList;
import java.util.List;
+import java.util.Optional;
import java.util.function.Supplier;
import java.util.logging.Logger;
@@ -20,14 +23,25 @@ public class GlobalPhaseRanker {
@Inject
public GlobalPhaseRanker(RankProfilesEvaluatorFactory factory) {
this.factory = factory;
- logger.info("using factory: " + factory);
+ logger.fine(() -> "Using factory: " + factory);
}
- public void process(Query query, Result result, String schema) {
- String rankProfile = query.getRanking().getProfile();
- GlobalPhaseData data = factory.evaluatorForSchema(schema)
- .flatMap(evaluator -> evaluator.getGlobalPhaseData(rankProfile))
- .orElse(null);
+ public Optional<ErrorMessage> validateNoSorting(Query query, String schema) {
+ var data = globalPhaseDataFor(query, schema).orElse(null);
+ if (data == null) return Optional.empty();
+ var sorting = query.getRanking().getSorting();
+ if (sorting == null || sorting.fieldOrders() == null) return Optional.empty();
+ for (var fieldOrder : sorting.fieldOrders()) {
+ if (!fieldOrder.getSorter().getName().equals("[rank]")
+ || fieldOrder.getSortOrder() != Sorting.Order.DESCENDING) {
+ return Optional.of(ErrorMessage.createIllegalQuery("Sorting is not supported with global phase"));
+ }
+ }
+ return Optional.empty();
+ }
+
+ public void rerankHits(Query query, Result result, String schema) {
+ var data = globalPhaseDataFor(query, schema).orElse(null);
if (data == null) return;
var functionEvaluatorSource = data.functionEvaluatorSource();
var prepared = findFromQuery(query, data.needInputs());
@@ -45,6 +59,11 @@ public class GlobalPhaseRanker {
ResultReranker.rerankHits(result, new HitRescorer(supplier), rerankCount);
}
+ private Optional<GlobalPhaseData> globalPhaseDataFor(Query query, String schema) {
+ return factory.evaluatorForSchema(schema)
+ .flatMap(evaluator -> evaluator.getGlobalPhaseData(query.getRanking().getProfile()));
+ }
+
record NameAndValue(String name, Tensor value) { }
/* do this only once per query: */
diff --git a/container-search/src/main/java/com/yahoo/search/rendering/JsonRenderer.java b/container-search/src/main/java/com/yahoo/search/rendering/JsonRenderer.java
index 44620179c1d..b36c8788877 100644
--- a/container-search/src/main/java/com/yahoo/search/rendering/JsonRenderer.java
+++ b/container-search/src/main/java/com/yahoo/search/rendering/JsonRenderer.java
@@ -76,10 +76,10 @@ import static com.fasterxml.jackson.databind.SerializationFeature.FLUSH_AFTER_WR
// NOTE: The JSON format is a public API. If new elements are added be sure to update the reference doc.
public class JsonRenderer extends AsynchronousSectionedRenderer<Result> {
- private static final CompoundName WRAP_DEEP_MAPS = new CompoundName("renderer.json.jsonMaps");
- private static final CompoundName WRAP_WSETS = new CompoundName("renderer.json.jsonWsets");
- private static final CompoundName DEBUG_RENDERING_KEY = new CompoundName("renderer.json.debug");
- private static final CompoundName JSON_CALLBACK = new CompoundName("jsoncallback");
+ private static final CompoundName WRAP_DEEP_MAPS = CompoundName.from("renderer.json.jsonMaps");
+ private static final CompoundName WRAP_WSETS = CompoundName.from("renderer.json.jsonWsets");
+ private static final CompoundName DEBUG_RENDERING_KEY = CompoundName.from("renderer.json.debug");
+ private static final CompoundName JSON_CALLBACK = CompoundName.from("jsoncallback");
// if this must be optimized, simply use com.fasterxml.jackson.core.SerializableString
private static final String BUCKET_LIMITS = "limits";
diff --git a/container-search/src/main/java/com/yahoo/search/searchers/CacheControlSearcher.java b/container-search/src/main/java/com/yahoo/search/searchers/CacheControlSearcher.java
index a4f03b088f2..29e8afa41ac 100644
--- a/container-search/src/main/java/com/yahoo/search/searchers/CacheControlSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/searchers/CacheControlSearcher.java
@@ -32,9 +32,9 @@ import com.yahoo.search.searchchain.Execution;
*/
public class CacheControlSearcher extends Searcher {
- private static final CompoundName cachecontrolNocache=new CompoundName("cachecontrol.nocache");
- private static final CompoundName cachecontrolMaxage=new CompoundName("cachecontrol.maxage");
- private static final CompoundName cachecontrolStaleage=new CompoundName("cachecontrol.staleage");
+ private static final CompoundName cachecontrolNocache=CompoundName.from("cachecontrol.nocache");
+ private static final CompoundName cachecontrolMaxage=CompoundName.from("cachecontrol.maxage");
+ private static final CompoundName cachecontrolStaleage=CompoundName.from("cachecontrol.staleage");
public static final String CACHE_CONTROL_HEADER = "Cache-Control";
diff --git a/container-search/src/main/java/com/yahoo/search/searchers/ConnectionControlSearcher.java b/container-search/src/main/java/com/yahoo/search/searchers/ConnectionControlSearcher.java
index 90cb05be1f8..18588b2026d 100644
--- a/container-search/src/main/java/com/yahoo/search/searchers/ConnectionControlSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/searchers/ConnectionControlSearcher.java
@@ -49,7 +49,7 @@ public class ConnectionControlSearcher extends Searcher {
private final LongSupplier clock;
- private static final CompoundName KEEPALIVE_MAXLIFETIMESECONDS = new CompoundName("connectioncontrol.maxlifetime");
+ private static final CompoundName KEEPALIVE_MAXLIFETIMESECONDS = CompoundName.from("connectioncontrol.maxlifetime");
private static final String HTTP_CONNECTION_HEADER_NAME = "Connection";
private static final String HTTP_CONNECTION_CLOSE_ARGUMENT = "Close";
diff --git a/container-search/src/main/java/com/yahoo/search/searchers/RateLimitingSearcher.java b/container-search/src/main/java/com/yahoo/search/searchers/RateLimitingSearcher.java
index cf81f62f64d..35a3c86f763 100755
--- a/container-search/src/main/java/com/yahoo/search/searchers/RateLimitingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/search/searchers/RateLimitingSearcher.java
@@ -54,11 +54,11 @@ public class RateLimitingSearcher extends Searcher {
/** Constant containing the name this Provides - "rateLimiting", for ordering constraints */
public static final String RATE_LIMITING = "rateLimiting";
- public static final CompoundName idKey = new CompoundName("rate.id");
- public static final CompoundName costKey = new CompoundName("rate.cost");
- public static final CompoundName quotaKey = new CompoundName("rate.quota");
- public static final CompoundName idDimensionKey = new CompoundName("rate.idDimension");
- public static final CompoundName dryRunKey = new CompoundName("rate.dryRun");
+ public static final CompoundName idKey = CompoundName.from("rate.id");
+ public static final CompoundName costKey = CompoundName.from("rate.cost");
+ public static final CompoundName quotaKey = CompoundName.from("rate.quota");
+ public static final CompoundName idDimensionKey = CompoundName.from("rate.idDimension");
+ public static final CompoundName dryRunKey = CompoundName.from("rate.dryRun");
private static final String requestsOverQuotaMetricName = "requestsOverQuota";
diff --git a/container-search/src/main/java/com/yahoo/search/yql/FieldFiller.java b/container-search/src/main/java/com/yahoo/search/yql/FieldFiller.java
index 833c1251a7b..6961a189d22 100644
--- a/container-search/src/main/java/com/yahoo/search/yql/FieldFiller.java
+++ b/container-search/src/main/java/com/yahoo/search/yql/FieldFiller.java
@@ -27,7 +27,7 @@ public class FieldFiller extends Searcher {
private final Set<String> intersectionOfAttributes;
private final SchemaInfo schemaInfo;
- public static final CompoundName FIELD_FILLER_DISABLE = new CompoundName("FieldFiller.disable");
+ public static final CompoundName FIELD_FILLER_DISABLE = CompoundName.from("FieldFiller.disable");
public FieldFiller(SchemaInfo schemaInfo) {
this.schemaInfo = schemaInfo;
diff --git a/container-search/src/main/java/com/yahoo/search/yql/FieldFilter.java b/container-search/src/main/java/com/yahoo/search/yql/FieldFilter.java
index 8cc6a1b42b9..df9722d3214 100644
--- a/container-search/src/main/java/com/yahoo/search/yql/FieldFilter.java
+++ b/container-search/src/main/java/com/yahoo/search/yql/FieldFilter.java
@@ -2,13 +2,11 @@
package com.yahoo.search.yql;
import java.util.Iterator;
-import java.util.Map.Entry;
import java.util.Set;
import com.yahoo.api.annotations.Beta;
import com.yahoo.component.chain.dependencies.After;
import com.yahoo.component.chain.dependencies.Before;
-import com.yahoo.prelude.fastsearch.FastHit;
import com.yahoo.processing.request.CompoundName;
import com.yahoo.search.Query;
import com.yahoo.search.Result;
@@ -27,7 +25,7 @@ import com.yahoo.search.searchchain.Execution;
@Before("com.yahoo.search.yql.FieldFiller")
public class FieldFilter extends Searcher {
- public static final CompoundName FIELD_FILTER_DISABLE = new CompoundName("FieldFilter.disable");
+ public static final CompoundName FIELD_FILTER_DISABLE = CompoundName.from("FieldFilter.disable");
/** Fields that should be kept even if not explicitly requested */
private static final Set<String> syntheticFields = Set.of("matchfeatures", "rankfeatures", "summaryfeatures");
diff --git a/container-search/src/main/java/com/yahoo/search/yql/MinimalQueryInserter.java b/container-search/src/main/java/com/yahoo/search/yql/MinimalQueryInserter.java
index e844bac21e8..ee4f931d532 100644
--- a/container-search/src/main/java/com/yahoo/search/yql/MinimalQueryInserter.java
+++ b/container-search/src/main/java/com/yahoo/search/yql/MinimalQueryInserter.java
@@ -39,10 +39,10 @@ public class MinimalQueryInserter extends Searcher {
public static final String EXTERNAL_YQL = "ExternalYql";
- public static final CompoundName YQL = new CompoundName("yql");
+ public static final CompoundName YQL = CompoundName.from("yql");
- private static final CompoundName MAX_HITS = new CompoundName("maxHits");
- private static final CompoundName MAX_OFFSET = new CompoundName("maxOffset");
+ private static final CompoundName MAX_HITS = CompoundName.from("maxHits");
+ private static final CompoundName MAX_OFFSET = CompoundName.from("maxOffset");
private static final Logger log = Logger.getLogger(MinimalQueryInserter.class.getName());
@Inject
diff --git a/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/MetricsSearcher.java b/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/MetricsSearcher.java
index ab9da8ccee5..536355ab62d 100644
--- a/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/MetricsSearcher.java
+++ b/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/MetricsSearcher.java
@@ -22,8 +22,8 @@ import static com.yahoo.vespa.streamingvisitors.VdsStreamingSearcher.STREAMING_S
*/
public class MetricsSearcher extends Searcher {
- private static final CompoundName metricsearcherId=new CompoundName("metricsearcher.id");
- private static final CompoundName streamingLoadtype=new CompoundName("streaming.loadtype");
+ private static final CompoundName metricsearcherId = CompoundName.from("metricsearcher.id");
+ private static final CompoundName streamingLoadtype = CompoundName.from("streaming.loadtype");
private static final Logger log = Logger.getLogger(MetricsSearcher.class.getName());
@@ -43,7 +43,8 @@ public class MetricsSearcher extends Searcher {
public Result search(Query query, Execution execution) {
long timeMs = System.currentTimeMillis();
- /** Backwards compatibility - convert metricsearcher.id to streaming.loadtype */
+ // Backwards compatibility - convert metricsearcher.id to streaming.loadtype
+ // TODO Cleanup at some point
String metricName = query.properties().getString(metricsearcherId);
if (metricName != null) {
query.properties().set(streamingLoadtype, metricName);
diff --git a/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java b/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java
index ca199f37dd7..524001748c5 100644
--- a/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java
@@ -45,9 +45,9 @@ import java.util.logging.Logger;
*/
public class VdsStreamingSearcher extends VespaBackEndSearcher {
- private static final CompoundName streamingUserid = new CompoundName("streaming.userid");
- private static final CompoundName streamingGroupname = new CompoundName("streaming.groupname");
- private static final CompoundName streamingSelection = new CompoundName("streaming.selection");
+ private static final CompoundName streamingUserid = CompoundName.from("streaming.userid");
+ private static final CompoundName streamingGroupname = CompoundName.from("streaming.groupname");
+ private static final CompoundName streamingSelection = CompoundName.from("streaming.selection");
static final String STREAMING_STATISTICS = "streaming.statistics";
private final VisitorFactory visitorFactory;
diff --git a/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsVisitor.java b/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsVisitor.java
index 85ef10b29f6..70dff6730ff 100644
--- a/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsVisitor.java
+++ b/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsVisitor.java
@@ -51,13 +51,13 @@ import java.util.logging.Level;
*/
class VdsVisitor extends VisitorDataHandler implements Visitor {
- private static final CompoundName streamingUserid=new CompoundName("streaming.userid");
- private static final CompoundName streamingGroupname=new CompoundName("streaming.groupname");
- private static final CompoundName streamingSelection=new CompoundName("streaming.selection");
- private static final CompoundName streamingFromtimestamp=new CompoundName("streaming.fromtimestamp");
- private static final CompoundName streamingTotimestamp=new CompoundName("streaming.totimestamp");
- private static final CompoundName streamingPriority=new CompoundName("streaming.priority");
- private static final CompoundName streamingMaxbucketspervisitor=new CompoundName("streaming.maxbucketspervisitor");
+ private static final CompoundName streamingUserid = CompoundName.from("streaming.userid");
+ private static final CompoundName streamingGroupname = CompoundName.from("streaming.groupname");
+ private static final CompoundName streamingSelection = CompoundName.from("streaming.selection");
+ private static final CompoundName streamingFromtimestamp = CompoundName.from("streaming.fromtimestamp");
+ private static final CompoundName streamingTotimestamp = CompoundName.from("streaming.totimestamp");
+ private static final CompoundName streamingPriority = CompoundName.from("streaming.priority");
+ private static final CompoundName streamingMaxbucketspervisitor = CompoundName.from("streaming.maxbucketspervisitor");
protected static final int MAX_BUCKETS_PER_VISITOR = 1024;
diff --git a/container-search/src/test/java/com/yahoo/prelude/query/parser/test/TokenizerTestCase.java b/container-search/src/test/java/com/yahoo/prelude/query/parser/test/TokenizerTestCase.java
index 1ff5574ec03..3a6be1521e2 100644
--- a/container-search/src/test/java/com/yahoo/prelude/query/parser/test/TokenizerTestCase.java
+++ b/container-search/src/test/java/com/yahoo/prelude/query/parser/test/TokenizerTestCase.java
@@ -13,7 +13,6 @@ import com.yahoo.prelude.query.parser.Tokenizer;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.List;
import static com.yahoo.prelude.query.parser.Token.Kind.COLON;
@@ -29,7 +28,9 @@ import static com.yahoo.prelude.query.parser.Token.Kind.SPACE;
import static com.yahoo.prelude.query.parser.Token.Kind.STAR;
import static com.yahoo.prelude.query.parser.Token.Kind.UNDERSCORE;
import static com.yahoo.prelude.query.parser.Token.Kind.WORD;
-import static org.junit.jupiter.api.Assertions.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Tests the tokenizer
@@ -283,7 +284,7 @@ public class TokenizerTestCase {
sd.addIndex(index2);
IndexFacts facts = new IndexFacts(new IndexModel(sd));
- IndexFacts.Session session = facts.newSession(Collections.emptySet(), Collections.emptySet());
+ IndexFacts.Session session = facts.newSession();
Tokenizer tokenizer = new Tokenizer(new SimpleLinguistics());
List<?> tokens = tokenizer.tokenize("normal a:b (normal testexact1:/,%#%&+-+ ) testexact2:ho_/&%&/()/aa*::*& b:c", "default", session);
// tokenizer.print();
@@ -328,7 +329,7 @@ public class TokenizerTestCase {
IndexFacts facts = new IndexFacts(new IndexModel(sd));
Tokenizer tokenizer = new Tokenizer(new SimpleLinguistics());
- IndexFacts.Session session = facts.newSession(Collections.emptySet(), Collections.emptySet());
+ IndexFacts.Session session = facts.newSession();
List<?> tokens = tokenizer.tokenize("normal a:b (normal testexact1:/,%#%&+-+ ) testexact2:ho_/&%&/()/aa*::*&", session);
assertEquals(new Token(WORD, "normal"), tokens.get(0));
assertEquals(new Token(SPACE, " "), tokens.get(1));
@@ -365,7 +366,7 @@ public class TokenizerTestCase {
IndexFacts facts = new IndexFacts(new IndexModel(sd));
Tokenizer tokenizer = new Tokenizer(new SimpleLinguistics());
- IndexFacts.Session session = facts.newSession(Collections.emptySet(), Collections.emptySet());
+ IndexFacts.Session session = facts.newSession();
List<?> tokens = tokenizer.tokenize("normal a:b (normal testexact1:/,%#%&+-+ ) testexact2:ho_/&%&/()/aa*::*", session);
assertEquals(new Token(WORD, "normal"), tokens.get(0));
assertEquals(new Token(SPACE, " "), tokens.get(1));
@@ -402,7 +403,7 @@ public class TokenizerTestCase {
IndexFacts facts = new IndexFacts(new IndexModel(sd));
Tokenizer tokenizer = new Tokenizer(new SimpleLinguistics());
- IndexFacts.Session session = facts.newSession(Collections.emptySet(), Collections.emptySet());
+ IndexFacts.Session session = facts.newSession();
List<?> tokens = tokenizer.tokenize("normal a:b (normal testexact1:!/%#%&+-+ ) testexact2:ho_/&%&/()/aa*::*&b:", session);
assertEquals(new Token(WORD, "normal"), tokens.get(0));
assertEquals(new Token(SPACE, " "), tokens.get(1));
@@ -439,7 +440,7 @@ public class TokenizerTestCase {
sd.addIndex(index2);
IndexFacts indexFacts = new IndexFacts(new IndexModel(sd));
- IndexFacts.Session facts = indexFacts.newSession(Collections.emptySet(), Collections.emptySet());
+ IndexFacts.Session facts = indexFacts.newSession();
Tokenizer tokenizer = new Tokenizer(new SimpleLinguistics());
List<?> tokens = tokenizer.tokenize("normal a:b (normal testexact1:foo) testexact2:bar", facts);
diff --git a/container-search/src/test/java/com/yahoo/prelude/test/IndexFactsTestCase.java b/container-search/src/test/java/com/yahoo/prelude/test/IndexFactsTestCase.java
index e6c5a18c9da..dbcb393c922 100644
--- a/container-search/src/test/java/com/yahoo/prelude/test/IndexFactsTestCase.java
+++ b/container-search/src/test/java/com/yahoo/prelude/test/IndexFactsTestCase.java
@@ -15,8 +15,12 @@ import org.junit.jupiter.api.Test;
import java.util.Collection;
import java.util.List;
import java.util.Map;
+import java.util.Set;
-import static org.junit.jupiter.api.Assertions.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Tests using synthetic index names for IndexFacts class.
@@ -180,7 +184,7 @@ public class IndexFactsTestCase {
query.getModel().getSources().add("one");
query.getModel().getRestrict().add("two");
- IndexFacts.Session indexFacts = createIndexFacts().newSession(List.of("clusterOne"), List.of());
+ IndexFacts.Session indexFacts = createIndexFacts().newSession(List.of("clusterOne"), Set.of());
assertTrue(indexFacts.isIndex("a"));
assertFalse(indexFacts.isIndex("b"));
assertTrue(indexFacts.isIndex("d"));
diff --git a/container-search/src/test/java/com/yahoo/search/query/profile/config/test/XmlReadingTestCase.java b/container-search/src/test/java/com/yahoo/search/query/profile/config/test/XmlReadingTestCase.java
index dcb41dc5e31..326c7985a5f 100644
--- a/container-search/src/test/java/com/yahoo/search/query/profile/config/test/XmlReadingTestCase.java
+++ b/container-search/src/test/java/com/yahoo/search/query/profile/config/test/XmlReadingTestCase.java
@@ -3,10 +3,8 @@ package com.yahoo.search.query.profile.config.test;
import com.yahoo.jdisc.http.HttpRequest.Method;
import com.yahoo.container.jdisc.HttpRequest;
-import com.yahoo.language.Language;
import com.yahoo.language.process.Embedder;
import com.yahoo.processing.request.CompoundName;
-import com.yahoo.search.query.profile.types.test.QueryProfileTypeTestCase;
import com.yahoo.tensor.Tensor;
import com.yahoo.tensor.TensorType;
import com.yahoo.yolean.Exceptions;
@@ -25,7 +23,11 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import static org.junit.jupiter.api.Assertions.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
/**
* @author bratseth
@@ -77,8 +79,8 @@ public class XmlReadingTestCase {
CompiledQueryProfile defaultProfile = cRegistry.getComponent("default");
assertNull(defaultProfile.getType());
assertEquals("20", defaultProfile.get("hits"));
- assertFalse(defaultProfile.isOverridable(new CompoundName("hits"), null));
- assertFalse(defaultProfile.isOverridable(new CompoundName("user.trusted"), null));
+ assertFalse(defaultProfile.isOverridable(CompoundName.from("hits"), null));
+ assertFalse(defaultProfile.isOverridable(CompoundName.from("user.trusted"), null));
assertEquals("false", defaultProfile.get("user.trusted"));
CompiledQueryProfile referencingProfile = cRegistry.getComponent("referencingModelSettings");
@@ -97,7 +99,7 @@ public class XmlReadingTestCase {
assertEquals("rootType", rootProfile.getType().getId().getName());
assertEquals(30, rootProfile.get("hits"));
//assertEquals(3, rootProfile.get("traceLevel"));
- assertTrue(rootProfile.isOverridable(new CompoundName("hits"), null));
+ assertTrue(rootProfile.isOverridable(CompoundName.from("hits"), null));
query = new Query(request, rootProfile);
assertEquals(3, query.getTrace().getLevel());
@@ -231,8 +233,8 @@ public class XmlReadingTestCase {
assertEquals("a.b.c-value", new Query("?d1=d1v", profile).properties().get("a.b.c"));
assertEquals("a.b.c-variant-value", new Query("?d1=d1v&d2=d2v", profile).properties().get("a.b.c"));
- assertTrue(profile.isOverridable(new CompoundName("a.b.c"), Map.of("d1", "d1v")));
- assertFalse(profile.isOverridable(new CompoundName("a.b.c"), Map.of("d1", "d1v", "d2", "d2v")));
+ assertTrue(profile.isOverridable(CompoundName.from("a.b.c"), Map.of("d1", "d1v")));
+ assertFalse(profile.isOverridable(CompoundName.from("a.b.c"), Map.of("d1", "d1v", "d2", "d2v")));
}
@Test
@@ -479,18 +481,18 @@ public class XmlReadingTestCase {
QueryProfileType type1 = registry.getTypeRegistry().getComponent("type1");
assertEquals(TensorType.fromSpec("tensor<float>(x[1])"),
- type1.getFieldType(new CompoundName("ranking.features.query(tensor_1)")).asTensorType());
- assertNull(type1.getFieldType(new CompoundName("ranking.features.query(tensor_2)")));
- assertNull(type1.getFieldType(new CompoundName("ranking.features.query(tensor_3)")));
+ type1.getFieldType(CompoundName.from("ranking.features.query(tensor_1)")).asTensorType());
+ assertNull(type1.getFieldType(CompoundName.from("ranking.features.query(tensor_2)")));
+ assertNull(type1.getFieldType(CompoundName.from("ranking.features.query(tensor_3)")));
assertEquals(TensorType.fromSpec("tensor(key{})"),
- type1.getFieldType(new CompoundName("ranking.features.query(tensor_4)")).asTensorType());
+ type1.getFieldType(CompoundName.from("ranking.features.query(tensor_4)")).asTensorType());
QueryProfileType type2 = registry.getTypeRegistry().getComponent("type2");
- assertNull(type2.getFieldType(new CompoundName("ranking.features.query(tensor_1)")));
+ assertNull(type2.getFieldType(CompoundName.from("ranking.features.query(tensor_1)")));
assertEquals(TensorType.fromSpec("tensor<float>(x[2])"),
- type2.getFieldType(new CompoundName("ranking.features.query(tensor_2)")).asTensorType());
+ type2.getFieldType(CompoundName.from("ranking.features.query(tensor_2)")).asTensorType());
assertEquals(TensorType.fromSpec("tensor<float>(x[3])"),
- type2.getFieldType(new CompoundName("ranking.features.query(tensor_3)")).asTensorType());
+ type2.getFieldType(CompoundName.from("ranking.features.query(tensor_3)")).asTensorType());
Query queryProfile1 = new Query.Builder().setQueryProfile(registry.getComponent("profile1"))
.setRequest("?query=test&ranking.features.query(tensor_1)=[1.200]")
diff --git a/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetInComplexStructureMicroBenchmark.java b/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetInComplexStructureMicroBenchmark.java
index cd5a8e08aa9..080aef25c5f 100644
--- a/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetInComplexStructureMicroBenchmark.java
+++ b/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetInComplexStructureMicroBenchmark.java
@@ -65,8 +65,8 @@ public class QueryProfileGetInComplexStructureMicroBenchmark {
Map<String,String> dimensionValues=createDimensionValueMap();
String prefix=generatePrefix();
final int dotInterval=1000000;
- final CompoundName found = new CompoundName(prefix + "a");
- final CompoundName notFound = new CompoundName(prefix + "nonexisting");
+ final CompoundName found = CompoundName.from(prefix + "a");
+ final CompoundName notFound = CompoundName.from(prefix + "nonexisting");
for (int i=0; i<count; i++) {
if (count>dotInterval && i%(dotInterval)==0)
System.out.print(".");
diff --git a/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetMicroBenchmark.java b/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetMicroBenchmark.java
index d38a1d64910..1123feb1b01 100644
--- a/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetMicroBenchmark.java
+++ b/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileGetMicroBenchmark.java
@@ -57,8 +57,8 @@ public class QueryProfileGetMicroBenchmark {
private void getValues(int count,Query query) {
final int dotInterval=10000000;
- CompoundName found = new CompoundName(propertyPrefix + "property1");
- CompoundName notFound = new CompoundName(propertyPrefix + "nonExisting");
+ CompoundName found = CompoundName.from(propertyPrefix + "property1");
+ CompoundName notFound = CompoundName.from(propertyPrefix + "nonExisting");
for (int i=0; i<count; i++) {
if (count>dotInterval && i%(count/dotInterval)==0)
System.out.print(".");
diff --git a/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileTestCase.java b/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileTestCase.java
index c4541fe9f58..7d5a2137770 100644
--- a/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileTestCase.java
+++ b/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileTestCase.java
@@ -486,7 +486,8 @@ public class QueryProfileTestCase {
assertEquals("a.b-value", cp.get("a.b", QueryProfileVariantsTestCase.toMap(p, new String[]{"x1"})));
}
- public void testSettingNonLeaf4b() {
+ @Test
+ void testSettingNonLeaf4b() {
QueryProfile p = new QueryProfile("test");
p.setDimensions(new String[] {"x"});
p.set("a","a-value", (QueryProfileRegistry)null);
@@ -533,7 +534,7 @@ public class QueryProfileTestCase {
p.set("a.b", "a.b-value", null);
{
- Map<String, ValueWithSource> values = p.compile(null).listValuesWithSources(new CompoundName(""), new HashMap<>(), null);
+ Map<String, ValueWithSource> values = p.compile(null).listValuesWithSources(CompoundName.empty, new HashMap<>(), null);
assertEquals(2, values.size());
assertEquals("a-value", values.get("a").value());
assertEquals("test", values.get("a").source());
@@ -542,7 +543,7 @@ public class QueryProfileTestCase {
}
{
- Map<String, ValueWithSource> values = p.compile(null).listValuesWithSources(new CompoundName("a"), new HashMap<>(), null);
+ Map<String, ValueWithSource> values = p.compile(null).listValuesWithSources(CompoundName.from("a"), new HashMap<>(), null);
assertEquals(1, values.size());
assertEquals("a.b-value", values.get("b").value());
assertEquals("test", values.get("b").source());
@@ -648,6 +649,32 @@ public class QueryProfileTestCase {
assertTrue(traceContains("foo: value", query));
}
+ @Test
+ void benchQueryCreation() throws InterruptedException {
+ QueryProfile p = new QueryProfile("test");
+ p.setDimensions(new String[]{"x", "y"});
+ p.set("clustering.something", "bar", null);
+ p.set("clustering.something", "bar", new String[]{"x1", "y1"}, null);
+ p.freeze();
+ CompiledQueryProfile cqp = p.compile(null);
+ var httpRequest = HttpRequest.createTestRequest("?x=x1&y=y1&query=bar&clustering.timeline.kano=tur&" +
+ "clustering.enable=true&clustering.timeline.bucketspec=-" +
+ "7d/3h&clustering.timeline.tophit=false&clustering.timeli" +
+ "ne=true", Method.GET);
+ for (int i = 0; i < 30000; i++) {
+ Query q = new Query(httpRequest, cqp);
+ assertTrue(q.properties().getBoolean(CompoundName.from("clustering.timeline"), false));
+ }
+ Thread.sleep(2000);
+ long start = System.nanoTime();
+ for (int i = 0; i < 100000; i++) {
+ Query q = new Query(httpRequest, cqp);
+ assertTrue(q.properties().getBoolean(CompoundName.from("clustering.timeline"), false));
+ }
+ long now = System.nanoTime();
+ System.out.println("Duration = " + (now - start)/1_000_000 + " ms");
+ }
+
// NB: NOT RECURSIVE
private boolean traceContains(String string, Query query) {
for (TraceNode node : query.getContext(true).getTrace().traceNode().children())
diff --git a/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileVariantsTestCase.java b/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileVariantsTestCase.java
index 1a6cfee75a5..36ff33ffd97 100644
--- a/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileVariantsTestCase.java
+++ b/container-search/src/test/java/com/yahoo/search/query/profile/test/QueryProfileVariantsTestCase.java
@@ -5,7 +5,6 @@ import ai.vespa.cloud.ApplicationId;
import ai.vespa.cloud.Environment;
import ai.vespa.cloud.Zone;
import ai.vespa.cloud.ZoneInfo;
-import com.yahoo.jdisc.application.Application;
import com.yahoo.jdisc.http.HttpRequest.Method;
import com.yahoo.container.jdisc.HttpRequest;
import com.yahoo.processing.request.CompoundName;
@@ -28,7 +27,9 @@ import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
-import static org.junit.jupiter.api.Assertions.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* @author bratseth
@@ -133,7 +134,7 @@ public class QueryProfileVariantsTestCase {
base.set("a.b", 1, new String[]{null, null, "d3-val"}, registry);
QueryProfileVariant aVariants = base.getVariants().getVariants().get(0);
assertEquals("[d1, d2, d3]",
- ((QueryProfile) base.getVariants().getVariants().get(0).values().get("a")).getDimensions().toString(),
+ ((QueryProfile) aVariants.values().get("a")).getDimensions().toString(),
"Variant dimensions are not overridden by the referenced dimensions");
}
@@ -184,7 +185,7 @@ public class QueryProfileVariantsTestCase {
{
Map<String, ValueWithSource> values = cRegistry.findQueryProfile("test")
- .listValuesWithSources(new CompoundName(""),
+ .listValuesWithSources(CompoundName.empty,
new HashMap<>(),
null);
assertEquals(1, values.size());
@@ -195,7 +196,7 @@ public class QueryProfileVariantsTestCase {
{
Map<String, ValueWithSource> values = cRegistry.findQueryProfile("test")
- .listValuesWithSources(new CompoundName(""),
+ .listValuesWithSources(CompoundName.empty,
toMap("x=x1", "y=y1", "z=z1"),
null);
assertEquals(2, values.size());
@@ -1456,7 +1457,7 @@ public class QueryProfileVariantsTestCase {
return context;
}
- public static final Map<String, String> toMap(String... bindings) {
+ public static Map<String, String> toMap(String... bindings) {
Map<String, String> context = new HashMap<>();
for (String binding : bindings) {
String[] entry = binding.split("=");
diff --git a/container-search/src/test/java/com/yahoo/search/query/profile/types/test/NameTestCase.java b/container-search/src/test/java/com/yahoo/search/query/profile/types/test/NameTestCase.java
index 37991f7f14f..982ebd80fbd 100644
--- a/container-search/src/test/java/com/yahoo/search/query/profile/types/test/NameTestCase.java
+++ b/container-search/src/test/java/com/yahoo/search/query/profile/types/test/NameTestCase.java
@@ -55,7 +55,7 @@ public class NameTestCase {
@Test
void testComponentIdAsCompoundName() {
String name = "a/b";
- assertEquals(new CompoundName(name), new QueryProfileType(name).getComponentIdAsCompoundName());
+ assertEquals(CompoundName.from(name), new QueryProfileType(name).getComponentIdAsCompoundName());
}
private void assertLegalName(String name) {
@@ -89,10 +89,6 @@ public class NameTestCase {
}
}
- private void assertIllegalFieldName(String name) {
- assertIllegalFieldName(name,"Could not set '" + name + "' to 'anyValue'","Illegal name '" + name + "'");
- }
-
/** Checks that this is illegal both for profiles and types */
private void assertIllegalFieldName(String name, String expectedHighError, String expectedLowError) {
try {
diff --git a/container-search/src/test/java/com/yahoo/search/query/profile/types/test/QueryProfileTypeTestCase.java b/container-search/src/test/java/com/yahoo/search/query/profile/types/test/QueryProfileTypeTestCase.java
index 21a9b2fe399..4000c48bfed 100644
--- a/container-search/src/test/java/com/yahoo/search/query/profile/types/test/QueryProfileTypeTestCase.java
+++ b/container-search/src/test/java/com/yahoo/search/query/profile/types/test/QueryProfileTypeTestCase.java
@@ -761,7 +761,7 @@ public class QueryProfileTypeTestCase {
}
private void assertNotPermitted(QueryProfile profile,String name,Object value) {
- String localName = new CompoundName(name).last();
+ String localName = CompoundName.from(name).last();
try {
profile.set(name, value, registry);
fail("Should fail setting " + name + " to " + value);
diff --git a/container-search/src/test/java/com/yahoo/search/query/properties/test/SubPropertiesTestCase.java b/container-search/src/test/java/com/yahoo/search/query/properties/SubPropertiesTestCase.java
index 0e9bf8d41ec..6b25dc526d9 100644
--- a/container-search/src/test/java/com/yahoo/search/query/properties/test/SubPropertiesTestCase.java
+++ b/container-search/src/test/java/com/yahoo/search/query/properties/SubPropertiesTestCase.java
@@ -1,5 +1,5 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.search.query.properties.test;
+package com.yahoo.search.query.properties;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
@@ -9,11 +9,12 @@ import java.util.HashSet;
import com.yahoo.processing.request.properties.PropertyMap;
import org.junit.jupiter.api.Test;
-import com.yahoo.search.query.properties.SubProperties;
/**
* @author <a href="mailto:arnebef@yahoo-inc.com">Arne Bergene Fossaa</a>
*/
+@SuppressWarnings({"removal"})
+
public class SubPropertiesTestCase {
@Test
diff --git a/container-search/src/test/java/com/yahoo/search/test/QueryTestCase.java b/container-search/src/test/java/com/yahoo/search/test/QueryTestCase.java
index 559b814a265..63655da0784 100644
--- a/container-search/src/test/java/com/yahoo/search/test/QueryTestCase.java
+++ b/container-search/src/test/java/com/yahoo/search/test/QueryTestCase.java
@@ -2,7 +2,6 @@
package com.yahoo.search.test;
import com.yahoo.component.chain.Chain;
-import com.yahoo.data.JsonProducer;
import com.yahoo.language.Language;
import com.yahoo.language.Linguistics;
import com.yahoo.language.detect.Detection;
@@ -16,7 +15,6 @@ import com.yahoo.prelude.Index;
import com.yahoo.prelude.IndexFacts;
import com.yahoo.prelude.IndexModel;
import com.yahoo.prelude.SearchDefinition;
-import com.yahoo.prelude.fastsearch.FastHit;
import com.yahoo.prelude.query.AndItem;
import com.yahoo.prelude.query.AndSegmentItem;
import com.yahoo.prelude.query.CompositeItem;
@@ -145,7 +143,7 @@ public class QueryTestCase {
@Test
void testCloneWithConnectivity() {
List<String> l = List.of("a", "b", "c", "a");
- printIt(l.stream().filter(i -> isA(i)).toList());
+ printIt(l.stream().filter(this::isA).toList());
printIt(l.stream().filter(i -> !isA(i)).toList());
Query q = new Query();
@@ -314,7 +312,7 @@ public class QueryTestCase {
profile.set("myField", "Profile: %{queryProfile}", null);
Query query = new Query(QueryTestCase.httpEncode("/search?queryProfile=myProfile"), profile.compile(null));
- String source = query.properties().getInstance(com.yahoo.search.query.profile.QueryProfileProperties.class).getQueryProfile().listValuesWithSources(new CompoundName(""), query.getHttpRequest().propertyMap(), query.properties()).get("myField").source();
+ String source = query.properties().getInstance(com.yahoo.search.query.profile.QueryProfileProperties.class).getQueryProfile().listValuesWithSources(CompoundName.empty, query.getHttpRequest().propertyMap(), query.properties()).get("myField").source();
assertEquals("myProfile", source);
}
@@ -327,6 +325,14 @@ public class QueryTestCase {
}
@Test
+ void testBooleanParameterNoQueryProfile() {
+ QueryProfile profile = new QueryProfile("myProfile");
+ Query query = new Query("/?query=something&ranking.softtimeout.enable=false");
+ assertFalse(query.properties().getBoolean("ranking.softtimeout.enable"));
+ assertFalse(query.getRanking().getSoftTimeout().getEnable());
+ }
+
+ @Test
void testQueryProfileSubstitution2() {
QueryProfile profile = new QueryProfile("myProfile");
profile.set("model.language", "en-US", null);
diff --git a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/zone/ZoneRegistry.java b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/zone/ZoneRegistry.java
index 4956dd475de..6cdb197b307 100644
--- a/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/zone/ZoneRegistry.java
+++ b/controller-api/src/main/java/com/yahoo/vespa/hosted/controller/api/integration/zone/ZoneRegistry.java
@@ -23,6 +23,7 @@ import com.yahoo.vespa.hosted.controller.api.integration.deployment.RunId;
import java.net.URI;
import java.time.Duration;
import java.util.List;
+import java.util.NoSuchElementException;
import java.util.Optional;
/**
@@ -50,6 +51,9 @@ public interface ZoneRegistry {
/** Returns the default region for the given environment, if one is configured */
Optional<RegionName> getDefaultRegion(Environment environment);
+ /** Throws {@link NoSuchElementException} if there is no such zone (in the system). */
+ ZoneApi get(ZoneId zoneId);
+
/** Returns the URI for the config server VIP in the given zone */
URI getConfigServerVipUri(ZoneId zoneId);
diff --git a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
index 6c614738ddf..8a0e2d01d8c 100644
--- a/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
+++ b/controller-server/src/main/java/com/yahoo/vespa/hosted/controller/restapi/application/ApplicationApiHandler.java
@@ -27,6 +27,7 @@ import com.yahoo.config.provision.NodeResources;
import com.yahoo.config.provision.TenantName;
import com.yahoo.config.provision.ZoneEndpoint.AllowedUrn;
import com.yahoo.config.provision.zone.RoutingMethod;
+import com.yahoo.config.provision.zone.ZoneApi;
import com.yahoo.config.provision.zone.ZoneId;
import com.yahoo.container.handler.metrics.JsonResponse;
import com.yahoo.container.jdisc.EmptyResponse;
@@ -1659,6 +1660,7 @@ public class ApplicationApiHandler extends AuditLoggingRequestHandler {
else {
deploymentObject.setString("environment", deployment.zone().environment().value());
deploymentObject.setString("region", deployment.zone().region().value());
+ addAvailabilityZone(deploymentObject, deployment.zone());
deploymentObject.setString("url", withPath(request.getUri().getPath() +
"/instance/" + instance.name().value() +
"/environment/" + deployment.zone().environment().value() +
@@ -1755,6 +1757,7 @@ public class ApplicationApiHandler extends AuditLoggingRequestHandler {
deploymentObject.setString("environment", deployment.zone().environment().value());
deploymentObject.setString("region", deployment.zone().region().value());
deploymentObject.setString("instance", instance.id().instance().value()); // pointless
+ addAvailabilityZone(deploymentObject, deployment.zone());
deploymentObject.setString("url", withPath(request.getUri().getPath() +
"/environment/" + deployment.zone().environment().value() +
"/region/" + deployment.zone().region().value(),
@@ -1837,6 +1840,7 @@ public class ApplicationApiHandler extends AuditLoggingRequestHandler {
response.setString("instance", deploymentId.applicationId().instance().value()); // pointless
response.setString("environment", deploymentId.zoneId().environment().value());
response.setString("region", deploymentId.zoneId().region().value());
+ addAvailabilityZone(response, deployment.zone());
var application = controller.applications().requireApplication(TenantAndApplicationId.from(deploymentId.applicationId()));
// Add zone endpoints
@@ -3020,6 +3024,12 @@ public class ApplicationApiHandler extends AuditLoggingRequestHandler {
return new MessageResponse("All deployments removed");
}
+ private void addAvailabilityZone(Cursor object, ZoneId zoneId) {
+ ZoneApi zone = controller.zoneRegistry().get(zoneId);
+ if (!zone.getCloudName().equals(CloudName.AWS)) return;
+ object.setString("availabilityZone", zone.getCloudNativeAvailabilityZone());
+ }
+
private ZoneId requireZone(String environment, String region) {
return requireZone(ZoneId.from(environment, region));
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneApiMock.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneApiMock.java
index 528ef6d6192..6fd44e09d8d 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneApiMock.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneApiMock.java
@@ -20,15 +20,17 @@ public class ZoneApiMock implements ZoneApi {
private final ZoneId virtualId;
private final CloudName cloudName;
private final String cloudNativeRegionName;
+ private final String cloudNativeAvailabilityZone;
public static Builder newBuilder() { return new Builder(); }
- private ZoneApiMock(SystemName systemName, ZoneId id, ZoneId virtualId, CloudName cloudName, String cloudNativeRegionName) {
+ private ZoneApiMock(SystemName systemName, ZoneId id, ZoneId virtualId, CloudName cloudName, String cloudNativeRegionName, String cloudNativeAvailabilityZone) {
this.systemName = systemName;
this.id = id;
this.virtualId = virtualId;
this.cloudName = cloudName;
this.cloudNativeRegionName = cloudNativeRegionName;
+ this.cloudNativeAvailabilityZone = cloudNativeAvailabilityZone;
if (virtualId != null && virtualId.equals(id)) {
throw new IllegalArgumentException("Virtual ID cannot be equal to zone ID: " + id);
}
@@ -64,6 +66,9 @@ public class ZoneApiMock implements ZoneApi {
public String getCloudNativeRegionName() { return cloudNativeRegionName; }
@Override
+ public String getCloudNativeAvailabilityZone() { return cloudNativeAvailabilityZone; }
+
+ @Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
@@ -88,6 +93,7 @@ public class ZoneApiMock implements ZoneApi {
private ZoneId virtualId = null;
private CloudName cloudName = CloudName.DEFAULT;
private String cloudNativeRegionName = id.region().value();
+ private String cloudNativeAvailabilityZone = "az1";
public Builder with(ZoneId id) {
this.id = id;
@@ -124,8 +130,13 @@ public class ZoneApiMock implements ZoneApi {
return this;
}
+ public Builder withCloudNativeAvailabilityZone(String cloudAvailabilityZone) {
+ this.cloudNativeAvailabilityZone = cloudAvailabilityZone;
+ return this;
+ }
+
public ZoneApiMock build() {
- return new ZoneApiMock(systemName, id, virtualId, cloudName, cloudNativeRegionName);
+ return new ZoneApiMock(systemName, id, virtualId, cloudName, cloudNativeRegionName, cloudNativeAvailabilityZone);
}
}
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java
index e59c677d0fa..611f0bab904 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/integration/ZoneRegistryMock.java
@@ -32,6 +32,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
@@ -62,9 +63,9 @@ public class ZoneRegistryMock extends AbstractComponent implements ZoneRegistry
if (system.isPublic()) {
this.zones = List.of(ZoneApiMock.fromId("test.us-east-1"),
ZoneApiMock.fromId("staging.us-east-3"),
- ZoneApiMock.fromId("prod.aws-us-east-1c"),
- ZoneApiMock.fromId("prod.aws-eu-west-1a"),
- ZoneApiMock.fromId("dev.aws-us-east-1c"));
+ ZoneApiMock.newBuilder().withId("prod.aws-us-east-1c").withCloud("aws").withCloudNativeAvailabilityZone("use1-az2").build(),
+ ZoneApiMock.newBuilder().withId("prod.aws-eu-west-1a").withCloud("aws").withCloudNativeAvailabilityZone("euw1-az3").build(),
+ ZoneApiMock.newBuilder().withId("dev.aws-us-east-1c").withCloud("aws").withCloudNativeAvailabilityZone("use1-az2").build());
setRoutingMethod(this.zones, RoutingMethod.exclusive);
} else {
this.zones = List.of(ZoneApiMock.fromId("test.us-east-1"),
@@ -299,6 +300,14 @@ public class ZoneRegistryMock extends AbstractComponent implements ZoneRegistry
}
@Override
+ public ZoneApi get(ZoneId zoneId) {
+ return zones.stream()
+ .filter(zone -> zone.getId().equals(zoneId))
+ .findFirst()
+ .orElseThrow(() -> new NoSuchElementException("No zone with id '" + zoneId + "'"));
+ }
+
+ @Override
public URI getMonitoringSystemUri(DeploymentId deploymentId) {
return URI.create("http://monitoring-system.test/?environment=" + deploymentId.zoneId().environment().value() + "&region="
+ deploymentId.zoneId().region().value() + "&application=" + deploymentId.applicationId().toShortString());
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ArchiveAccessMaintainerTest.java b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ArchiveAccessMaintainerTest.java
index 0490a9bdcc5..fe14d696011 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ArchiveAccessMaintainerTest.java
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/maintenance/ArchiveAccessMaintainerTest.java
@@ -43,10 +43,12 @@ public class ArchiveAccessMaintainerTest {
assertEquals(new ArchiveAccess().withAWSRole(tenant1role), archiveService.authorizeAccessByTenantName.get(tenant1));
assertEquals(new ArchiveAccess().withAWSRole(tenant2role), archiveService.authorizeAccessByTenantName.get(tenant2));
+ var zoneRegistry = tester.controller().zoneRegistry();
var expected = Map.of("archive.bucketCount",
- tester.controller().zoneRegistry().zonesIncludingSystem().all().ids().stream()
+ zoneRegistry.zonesIncludingSystem().all().ids().stream()
.collect(Collectors.toMap(
- zone -> Map.of("zone", zone.value(), "cloud", "default"),
+ zone -> Map.of("zone", zone.value(), "cloud",
+ zoneRegistry.hasZone(zone) ? zoneRegistry.get(zone).getCloudName().value() : "default"),
zone -> zone.equals(testZone) ? 1d : 0d)));
assertEquals(expected, metric.metrics());
diff --git a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-cloud.json b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-cloud.json
index 4d0f1259c07..bb4136ed0ba 100644
--- a/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-cloud.json
+++ b/controller-server/src/test/java/com/yahoo/vespa/hosted/controller/restapi/application/responses/deployment-cloud.json
@@ -4,6 +4,7 @@
"instance": "default",
"environment": "prod",
"region": "aws-us-east-1c",
+ "availabilityZone": "use1-az2",
"endpoints": [
{
"cluster": "default",
@@ -30,7 +31,7 @@
},
"status": "complete",
"quota": 1.304,
- "activity": { },
+ "activity": {},
"metrics": {
"queriesPerSecond": 0.0,
"writesPerSecond": 0.0,
diff --git a/flags/src/main/java/com/yahoo/vespa/flags/Flags.java b/flags/src/main/java/com/yahoo/vespa/flags/Flags.java
index 76702d3791d..4ba5911dd30 100644
--- a/flags/src/main/java/com/yahoo/vespa/flags/Flags.java
+++ b/flags/src/main/java/com/yahoo/vespa/flags/Flags.java
@@ -79,6 +79,12 @@ public class Flags {
" latency-amortized-over-requests, latency-amortized-over-time",
"Takes effect at redeployment (requires restart)",
ZONE_ID, APPLICATION_ID);
+ public static final UnboundStringFlag SUMMARY_DECODE_POLICY = defineStringFlag(
+ "summary-decode-policy", "eager",
+ List.of("baldersheim"), "2023-03-30", "2023-12-31",
+ "Select summary decoding policy, valid values are eager and on-demand/ondemand.",
+ "Takes effect at redeployment (requires restart)",
+ ZONE_ID, APPLICATION_ID);
public static final UnboundStringFlag FEED_SEQUENCER_TYPE = defineStringFlag(
"feed-sequencer-type", "THROUGHPUT",
diff --git a/parent/pom.xml b/parent/pom.xml
index b0ba6e89f6e..ace6a47d6bc 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -1170,7 +1170,7 @@
<org.lz4.version>1.8.0</org.lz4.version>
<prometheus.client.version>0.6.0</prometheus.client.version>
<protobuf.version>3.21.7</protobuf.version>
- <spifly.version>1.3.5</spifly.version>
+ <spifly.version>1.3.6</spifly.version>
<surefire.version>3.0.0-M9</surefire.version>
<wiremock.version>2.35.0</wiremock.version>
<zero-allocation-hashing.version>0.16</zero-allocation-hashing.version>
diff --git a/screwdriver.yaml b/screwdriver.yaml
index 417fafcc4bc..19fba104de6 100644
--- a/screwdriver.yaml
+++ b/screwdriver.yaml
@@ -250,7 +250,7 @@ jobs:
- update-sample-apps: |
screwdriver/update-vespa-version-in-sample-apps.sh $VESPA_VERSION
- update-released-time: |
- screwdriver/factory-command.sh update-released-time
+ screwdriver/factory-command.sh update-released-time $VESPA_VERSION
publish-legacy-release:
image: docker.io/vespaengine/vespa-build-centos-stream8:latest
diff --git a/screwdriver/release-container-image.sh b/screwdriver/release-container-image.sh
index 8b61dde4e1d..33231890626 100755
--- a/screwdriver/release-container-image.sh
+++ b/screwdriver/release-container-image.sh
@@ -57,7 +57,8 @@ for data in "Dockerfile vespa"; do
--jobs 2 \
--layers=false \
--manifest "vespaengine/$IMAGE_NAME:$VESPA_VERSION" \
- --platform linux/amd64,linux/arm64 | cat
+ --platform linux/amd64,linux/arm64 \
+ --squash | cat
# Test
buildah tag vespaengine/$IMAGE_NAME:$VESPA_VERSION vespaengine/$IMAGE_NAME:latest
diff --git a/searchcore/CMakeLists.txt b/searchcore/CMakeLists.txt
index 131460b0384..426e577b2d2 100644
--- a/searchcore/CMakeLists.txt
+++ b/searchcore/CMakeLists.txt
@@ -64,7 +64,6 @@ vespa_define_module(
src/tests/proton/attribute/attributes_state_explorer
src/tests/proton/attribute/document_field_extractor
src/tests/proton/attribute/document_field_populator
- src/tests/proton/attribute/exclusive_attribute_read_accessor
src/tests/proton/attribute/imported_attributes_context
src/tests/proton/attribute/imported_attributes_repo
src/tests/proton/bucketdb/bucketdb
diff --git a/searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp b/searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp
index c9f3a1c1de8..140012624c2 100644
--- a/searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp
+++ b/searchcore/src/tests/proton/attribute/attribute_manager/attribute_manager_test.cpp
@@ -5,7 +5,6 @@
#include <vespa/searchcore/proton/attribute/attribute_manager_reconfig.h>
#include <vespa/searchcore/proton/attribute/attribute_writer.h>
#include <vespa/searchcore/proton/attribute/attributemanager.h>
-#include <vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.h>
#include <vespa/searchcore/proton/attribute/imported_attributes_repo.h>
#include <vespa/searchcore/proton/attribute/sequential_attributes_initializer.h>
#include <vespa/searchcore/proton/bucketdb/bucket_db_owner.h>
@@ -755,15 +754,6 @@ TEST_F("require that we can call functions on all attributes via functor",
EXPECT_EQUAL("a1,a2,a3", functor->getSortedNames());
}
-TEST_F("require that we can acquire exclusive read access to attribute", Fixture)
-{
- f.addAttribute("attr");
- ExclusiveAttributeReadAccessor::UP attrAccessor = f._m.getExclusiveReadAccessor("attr");
- ExclusiveAttributeReadAccessor::UP noneAccessor = f._m.getExclusiveReadAccessor("none");
- EXPECT_TRUE(attrAccessor.get() != nullptr);
- EXPECT_TRUE(noneAccessor.get() == nullptr);
-}
-
TEST_F("require that imported attributes are exposed via attribute context together with regular attributes", Fixture)
{
f.addAttribute("attr");
diff --git a/searchcore/src/tests/proton/attribute/attributes_state_explorer/attributes_state_explorer_test.cpp b/searchcore/src/tests/proton/attribute/attributes_state_explorer/attributes_state_explorer_test.cpp
index bcd200cfd2f..aa4871d3a12 100644
--- a/searchcore/src/tests/proton/attribute/attributes_state_explorer/attributes_state_explorer_test.cpp
+++ b/searchcore/src/tests/proton/attribute/attributes_state_explorer/attributes_state_explorer_test.cpp
@@ -92,13 +92,13 @@ TEST_F(AttributesStateExplorerTest, require_that_attributes_are_exposed_as_child
{
StringVector children = _explorer.get_children_names();
std::sort(children.begin(), children.end());
- EXPECT_EQ(StringVector({"btree", "extra", "hash", "hybrid", "regular"}), children);
+ EXPECT_EQ(StringVector({"btree", "hash", "hybrid", "regular"}), children);
}
TEST_F(AttributesStateExplorerTest, require_that_attributes_are_explorable)
{
EXPECT_TRUE(_explorer.get_child("regular").get() != nullptr);
- EXPECT_TRUE(_explorer.get_child("extra").get() != nullptr);
+ EXPECT_TRUE(_explorer.get_child("extra").get() == nullptr);
EXPECT_TRUE(_explorer.get_child("not").get() == nullptr);
}
diff --git a/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/.gitignore b/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/.gitignore
deleted file mode 100644
index f3666eecb6e..00000000000
--- a/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-searchcore_exclusive_attribute_read_accessor_test_app
diff --git a/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/CMakeLists.txt b/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/CMakeLists.txt
deleted file mode 100644
index 981d2acd7c5..00000000000
--- a/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/CMakeLists.txt
+++ /dev/null
@@ -1,9 +0,0 @@
-# Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-vespa_add_executable(searchcore_exclusive_attribute_read_accessor_test_app TEST
- SOURCES
- exclusive_attribute_read_accessor_test.cpp
- DEPENDS
- searchcore_attribute
- searchcore_pcommon
-)
-vespa_add_test(NAME searchcore_exclusive_attribute_read_accessor_test_app COMMAND searchcore_exclusive_attribute_read_accessor_test_app)
diff --git a/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/exclusive_attribute_read_accessor_test.cpp b/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/exclusive_attribute_read_accessor_test.cpp
deleted file mode 100644
index 8b093be08b7..00000000000
--- a/searchcore/src/tests/proton/attribute/exclusive_attribute_read_accessor/exclusive_attribute_read_accessor_test.cpp
+++ /dev/null
@@ -1,55 +0,0 @@
-// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-#include <vespa/vespalib/testkit/testapp.h>
-
-#include <vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.h>
-#include <vespa/searchcommon/attribute/config.h>
-#include <vespa/searchlib/attribute/attributefactory.h>
-#include <vespa/searchlib/attribute/attributevector.h>
-#include <vespa/vespalib/util/sequencedtaskexecutor.h>
-#include <vespa/vespalib/util/gate.h>
-
-using namespace proton;
-using namespace search;
-using namespace vespalib;
-
-using ReadGuard = ExclusiveAttributeReadAccessor::Guard;
-VESPA_THREAD_STACK_TAG(test_executor)
-
-AttributeVector::SP
-createAttribute()
-{
- attribute::Config cfg(attribute::BasicType::INT32, attribute::CollectionType::SINGLE);
- return search::AttributeFactory::createAttribute("myattr", cfg);
-}
-
-struct Fixture
-{
- AttributeVector::SP attribute;
- std::unique_ptr<ISequencedTaskExecutor> writer;
- ExclusiveAttributeReadAccessor accessor;
-
- Fixture()
- : attribute(createAttribute()),
- writer(SequencedTaskExecutor::create(test_executor, 1)),
- accessor(attribute, *writer)
- {}
-};
-
-TEST_F("require that attribute write thread is blocked while guard is held", Fixture)
-{
- ReadGuard::UP guard = f.accessor.takeGuard();
- Gate gate;
- f.writer->execute(f.writer->getExecutorIdFromName(f.attribute->getNamePrefix()), [&gate]() { gate.countDown(); });
- bool reachedZero = gate.await(100ms);
- EXPECT_FALSE(reachedZero);
- EXPECT_EQUAL(1u, gate.getCount());
-
- guard.reset();
- gate.await();
- EXPECT_EQUAL(0u, gate.getCount());
-}
-
-TEST_MAIN()
-{
- TEST_RUN_ALL();
-}
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/CMakeLists.txt b/searchcore/src/vespa/searchcore/proton/attribute/CMakeLists.txt
index 82bb188870f..70a91b418a9 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/CMakeLists.txt
+++ b/searchcore/src/vespa/searchcore/proton/attribute/CMakeLists.txt
@@ -7,6 +7,7 @@ vespa_add_library(searchcore_attribute STATIC
attribute_collection_spec.cpp
attribute_config_inspector.cpp
attribute_directory.cpp
+ attribute_executor.cpp
attribute_factory.cpp
attribute_initializer.cpp
attribute_initializer_result.cpp
@@ -30,7 +31,6 @@ vespa_add_library(searchcore_attribute STATIC
document_field_extractor.cpp
document_field_populator.cpp
document_field_retriever.cpp
- exclusive_attribute_read_accessor.cpp
filter_attribute_manager.cpp
flushableattribute.cpp
imported_attributes_context.cpp
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/attribute_executor.cpp b/searchcore/src/vespa/searchcore/proton/attribute/attribute_executor.cpp
new file mode 100644
index 00000000000..4d854bfdca1
--- /dev/null
+++ b/searchcore/src/vespa/searchcore/proton/attribute/attribute_executor.cpp
@@ -0,0 +1,34 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#include "attribute_executor.h"
+#include "i_attribute_manager.h"
+#include <vespa/searchlib/attribute/attributevector.h>
+#include <vespa/vespalib/util/isequencedtaskexecutor.h>
+#include <future>
+
+using search::AttributeVector;
+
+namespace proton {
+
+AttributeExecutor::AttributeExecutor(std::shared_ptr<IAttributeManager> mgr,
+ std::shared_ptr<AttributeVector> attr)
+ : _mgr(std::move(mgr)),
+ _attr(std::move(attr))
+{
+}
+
+AttributeExecutor::~AttributeExecutor() = default;
+
+void
+AttributeExecutor::run_sync(std::function<void()> task) const
+{
+ vespalib::string name = _attr->getNamePrefix();
+ auto& writer = _mgr->getAttributeFieldWriter();
+ std::promise<void> promise;
+ auto future = promise.get_future();
+ auto id = writer.getExecutorIdFromName(name);
+ writer.execute(id, [&task, promise=std::move(promise)]() mutable { task(); promise.set_value(); });
+ future.wait();
+}
+
+} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/attribute_executor.h b/searchcore/src/vespa/searchcore/proton/attribute/attribute_executor.h
new file mode 100644
index 00000000000..4a13fe1d886
--- /dev/null
+++ b/searchcore/src/vespa/searchcore/proton/attribute/attribute_executor.h
@@ -0,0 +1,32 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+#include <vespa/vespalib/stllike/string.h>
+#include <functional>
+#include <memory>
+
+namespace search { class AttributeVector; }
+
+namespace proton {
+
+struct IAttributeManager;
+
+/*
+ * Class for executing task in attribute vector write thread.
+ */
+class AttributeExecutor
+{
+private:
+ std::shared_ptr<IAttributeManager> _mgr;
+ std::shared_ptr<search::AttributeVector> _attr;
+
+public:
+ AttributeExecutor(std::shared_ptr<IAttributeManager> mgr,
+ std::shared_ptr<search::AttributeVector> attr);
+ ~AttributeExecutor();
+ void run_sync(std::function<void()> task) const;
+ const search::AttributeVector& get_attr() const noexcept { return *_attr; }
+};
+
+} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/attribute_manager_explorer.cpp b/searchcore/src/vespa/searchcore/proton/attribute/attribute_manager_explorer.cpp
index b7d6cf9e87a..1ea02c729ff 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/attribute_manager_explorer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/attribute/attribute_manager_explorer.cpp
@@ -1,9 +1,11 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "attribute_manager_explorer.h"
+#include "attribute_executor.h"
#include "attribute_vector_explorer.h"
#include <vespa/searchlib/attribute/attributevector.h>
+using search::AttributeVector;
using vespalib::slime::Inserter;
namespace proton {
@@ -25,8 +27,7 @@ AttributeManagerExplorer::get_state(const Inserter &inserter, bool full) const
std::vector<vespalib::string>
AttributeManagerExplorer::get_children_names() const
{
- std::vector<search::AttributeGuard> attributes;
- _mgr->getAttributeListAll(attributes);
+ auto& attributes = _mgr->getWritableAttributes();
std::vector<vespalib::string> names;
for (const auto &attr : attributes) {
names.push_back(attr->getName());
@@ -37,11 +38,13 @@ AttributeManagerExplorer::get_children_names() const
std::unique_ptr<vespalib::StateExplorer>
AttributeManagerExplorer::get_child(vespalib::stringref name) const
{
- auto attr = _mgr->getExclusiveReadAccessor(name);
- if (attr.get() != nullptr) {
- return std::make_unique<AttributeVectorExplorer>(std::move(attr));
+ auto guard = _mgr->getAttribute(name);
+ auto attr = guard ? guard->getSP() : std::shared_ptr<AttributeVector>();
+ if (attr && _mgr->getWritableAttribute(name) != nullptr) {
+ auto executor = std::make_unique<AttributeExecutor>(_mgr, std::move(attr));
+ return std::make_unique<AttributeVectorExplorer>(std::move(executor));
}
- return std::unique_ptr<vespalib::StateExplorer>();
+ return {};
}
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.cpp b/searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.cpp
index c153f873480..6244bdbea33 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.cpp
@@ -1,6 +1,7 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "attribute_vector_explorer.h"
+#include "attribute_executor.h"
#include <vespa/searchlib/attribute/i_enum_store.h>
#include <vespa/searchlib/attribute/i_enum_store_dictionary.h>
#include <vespa/searchlib/attribute/multi_value_mapping.h>
@@ -14,9 +15,11 @@ using search::attribute::Status;
using search::AddressSpaceUsage;
using search::AttributeVector;
using search::IEnumStore;
+using search::StateExplorerUtils;
using vespalib::AddressSpace;
using vespalib::MemoryUsage;
using search::attribute::MultiValueMappingBase;
+using search::attribute::IAttributeVector;
using search::attribute::IPostingListAttributeBase;
using namespace vespalib::slime;
@@ -25,26 +28,6 @@ namespace proton {
namespace {
void
-convertStatusToSlime(const Status &status, Cursor &object)
-{
- object.setLong("numDocs", status.getNumDocs());
- object.setLong("numValues", status.getNumValues());
- object.setLong("numUniqueValues", status.getNumUniqueValues());
- object.setLong("lastSerialNum", status.getLastSyncToken());
- object.setLong("updateCount", status.getUpdateCount());
- object.setLong("nonIdempotentUpdateCount", status.getNonIdempotentUpdateCount());
- object.setLong("bitVectors", status.getBitVectors());
- {
- Cursor &memory = object.setObject("memoryUsage");
- memory.setLong("allocatedBytes", status.getAllocated());
- memory.setLong("usedBytes", status.getUsed());
- memory.setLong("deadBytes", status.getDead());
- memory.setLong("onHoldBytes", status.getOnHold());
- memory.setLong("onHoldBytesMax", status.getOnHoldMax());
- }
-}
-
-void
convertGenerationToSlime(const AttributeVector &attr, Cursor &object)
{
object.setLong("oldest_used", attr.get_oldest_used_generation());
@@ -71,7 +54,7 @@ convertAddressSpaceUsageToSlime(const AddressSpaceUsage &usage, Cursor &object)
void
convertMemoryUsageToSlime(const MemoryUsage &usage, Cursor &object)
{
- search::StateExplorerUtils::memory_usage_to_slime(usage, object);
+ StateExplorerUtils::memory_usage_to_slime(usage, object);
}
void
@@ -116,20 +99,25 @@ convertPostingBaseToSlime(const IPostingListAttributeBase &postingBase, Cursor &
}
-AttributeVectorExplorer::AttributeVectorExplorer(ExclusiveAttributeReadAccessor::UP attribute)
- : _attribute(std::move(attribute))
+AttributeVectorExplorer::AttributeVectorExplorer(std::unique_ptr<AttributeExecutor> executor)
+ : _executor(std::move(executor))
{
}
void
AttributeVectorExplorer::get_state(const vespalib::slime::Inserter &inserter, bool full) const
{
- ExclusiveAttributeReadAccessor::Guard::UP readGuard = _attribute->takeGuard();
- const AttributeVector &attr = readGuard->get();
+ auto& attr = _executor->get_attr();
+ _executor->run_sync([this, &attr, &inserter, full] { get_state_helper(attr, inserter, full); });
+}
+
+void
+AttributeVectorExplorer::get_state_helper(const AttributeVector& attr, const vespalib::slime::Inserter &inserter, bool full) const
+{
const Status &status = attr.getStatus();
Cursor &object = inserter.insertObject();
if (full) {
- convertStatusToSlime(status, object.setObject("status"));
+ StateExplorerUtils::status_to_slime(status, object.setObject("status"));
convertGenerationToSlime(attr, object.setObject("generation"));
convertAddressSpaceUsageToSlime(attr.getAddressSpaceUsage(), object.setObject("addressSpaceUsage"));
// TODO: Consider making enum store, multivalue mapping, posting list attribute and tensor attribute
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.h b/searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.h
index 204a81ed629..f7bcc98d321 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.h
+++ b/searchcore/src/vespa/searchcore/proton/attribute/attribute_vector_explorer.h
@@ -2,21 +2,25 @@
#pragma once
-#include "exclusive_attribute_read_accessor.h"
#include <vespa/vespalib/net/http/state_explorer.h>
+namespace search { class AttributeVector; }
+
namespace proton {
+class AttributeExecutor;
+
/**
* Class used to explore the state of an attribute vector.
*/
class AttributeVectorExplorer : public vespalib::StateExplorer
{
private:
- ExclusiveAttributeReadAccessor::UP _attribute;
+ std::unique_ptr<const AttributeExecutor> _executor;
+ void get_state_helper(const search::AttributeVector& attr, const vespalib::slime::Inserter &inserter, bool full) const;
public:
- AttributeVectorExplorer(ExclusiveAttributeReadAccessor::UP attribute);
+ AttributeVectorExplorer(std::unique_ptr<AttributeExecutor> executor);
// Implements vespalib::StateExplorer
void get_state(const vespalib::slime::Inserter &inserter, bool full) const override;
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/attributemanager.cpp b/searchcore/src/vespa/searchcore/proton/attribute/attributemanager.cpp
index 161a4294908..ee04f17d378 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/attributemanager.cpp
+++ b/searchcore/src/vespa/searchcore/proton/attribute/attributemanager.cpp
@@ -638,16 +638,6 @@ AttributeManager::asyncForAttribute(const vespalib::string &name, std::unique_pt
[attr=std::move(attrsp), func=std::move(func)]() { (*func)(*attr); });
}
-ExclusiveAttributeReadAccessor::UP
-AttributeManager::getExclusiveReadAccessor(const vespalib::string &name) const
-{
- AttributeVector::SP attribute = findAttribute(name);
- if (attribute) {
- return std::make_unique<ExclusiveAttributeReadAccessor>(attribute, _attributeFieldWriter);
- }
- return {};
-}
-
void
AttributeManager::setImportedAttributes(std::unique_ptr<ImportedAttributesRepo> attributes)
{
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/attributemanager.h b/searchcore/src/vespa/searchcore/proton/attribute/attributemanager.h
index c924c1c5b20..d6d0964f97e 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/attributemanager.h
+++ b/searchcore/src/vespa/searchcore/proton/attribute/attributemanager.h
@@ -182,8 +182,6 @@ public:
void asyncForEachAttribute(std::shared_ptr<IAttributeFunctor> func, OnDone onDone) const override;
void asyncForAttribute(const vespalib::string &name, std::unique_ptr<IAttributeFunctor> func) const override;
- ExclusiveAttributeReadAccessor::UP getExclusiveReadAccessor(const vespalib::string &name) const override;
-
void setImportedAttributes(std::unique_ptr<ImportedAttributesRepo> attributes) override;
const ImportedAttributesRepo *getImportedAttributes() const override { return _importedAttributes.get(); }
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.cpp b/searchcore/src/vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.cpp
deleted file mode 100644
index ed72418ced4..00000000000
--- a/searchcore/src/vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.cpp
+++ /dev/null
@@ -1,60 +0,0 @@
-// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-#include "exclusive_attribute_read_accessor.h"
-#include <vespa/vespalib/util/gate.h>
-#include <vespa/searchlib/attribute/attributevector.h>
-#include <vespa/vespalib/util/isequencedtaskexecutor.h>
-
-namespace proton {
-
-using search::AttributeVector;
-using vespalib::ISequencedTaskExecutor;
-using vespalib::Executor;
-using vespalib::Gate;
-
-using GateSP = std::shared_ptr<Gate>;
-
-ExclusiveAttributeReadAccessor::Guard::Guard(const AttributeVector &attribute,
- const GateSP &exitGate)
- : _attribute(attribute),
- _exitGate(exitGate)
-{
-}
-
-ExclusiveAttributeReadAccessor::Guard::~Guard()
-{
- _exitGate->countDown();
-}
-
-ExclusiveAttributeReadAccessor::
-ExclusiveAttributeReadAccessor(const AttributeVector::SP &attribute,
- ISequencedTaskExecutor &attributeFieldWriter)
- : _attribute(attribute),
- _attributeFieldWriter(attributeFieldWriter)
-{
-}
-
-namespace {
-
-void
-attributeWriteBlockingTask(AttributeVector::SP attribute, GateSP entranceGate, GateSP exitGate)
-{
- attribute->commit(true);
- entranceGate->countDown();
- exitGate->await();
-}
-
-}
-
-ExclusiveAttributeReadAccessor::Guard::UP
-ExclusiveAttributeReadAccessor::takeGuard()
-{
- GateSP entranceGate = std::make_shared<Gate>();
- GateSP exitGate = std::make_shared<Gate>();
- _attributeFieldWriter.execute(_attributeFieldWriter.getExecutorIdFromName(_attribute->getNamePrefix()),
- [this, entranceGate, exitGate]() { attributeWriteBlockingTask(_attribute, entranceGate, exitGate); });
- entranceGate->await();
- return std::make_unique<Guard>(*_attribute, exitGate);
-}
-
-} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.h b/searchcore/src/vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.h
deleted file mode 100644
index aa756764b1a..00000000000
--- a/searchcore/src/vespa/searchcore/proton/attribute/exclusive_attribute_read_accessor.h
+++ /dev/null
@@ -1,53 +0,0 @@
-// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-#pragma once
-
-#include <memory>
-
-namespace search {
- class AttributeVector;
-}
-namespace vespalib {
- class Gate;
- class ISequencedTaskExecutor;
-}
-
-namespace proton {
-
-/**
- * Class that provides exclusive read access to an attribute vector
- * while the write thread for that attribute is blocked.
- *
- * The attribute write thread is blocked while a guard is held.
- */
-class ExclusiveAttributeReadAccessor
-{
-public:
- class Guard
- {
- private:
- const search::AttributeVector &_attribute;
- std::shared_ptr<vespalib::Gate> _exitGate;
-
- public:
- using UP = std::unique_ptr<Guard>;
- Guard(const search::AttributeVector &attribute,
- const std::shared_ptr<vespalib::Gate> &exitGate);
- ~Guard();
- const search::AttributeVector &get() const { return _attribute; }
- };
-
-private:
- using AttributeVectorSP = std::shared_ptr<search::AttributeVector>;
- AttributeVectorSP _attribute;
- vespalib::ISequencedTaskExecutor &_attributeFieldWriter;
-
-public:
- using UP = std::unique_ptr<ExclusiveAttributeReadAccessor>;
-
- ExclusiveAttributeReadAccessor(const AttributeVectorSP &attribute,
- vespalib::ISequencedTaskExecutor &attributeFieldWriter);
- Guard::UP takeGuard();
-};
-
-} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.cpp b/searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.cpp
index 6f6cd01b4a4..6329c633727 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.cpp
+++ b/searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.cpp
@@ -232,12 +232,6 @@ FilterAttributeManager::asyncForAttribute(const vespalib::string &name, std::uni
}
-ExclusiveAttributeReadAccessor::UP
-FilterAttributeManager::getExclusiveReadAccessor(const vespalib::string &name) const
-{
- return (acceptAttribute(name)) ? _mgr->getExclusiveReadAccessor(name) : ExclusiveAttributeReadAccessor::UP();
-}
-
void
FilterAttributeManager::setImportedAttributes(std::unique_ptr<ImportedAttributesRepo>)
{
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.h b/searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.h
index dff18290330..6d5ff682ca1 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.h
+++ b/searchcore/src/vespa/searchcore/proton/attribute/filter_attribute_manager.h
@@ -52,7 +52,6 @@ public:
const std::vector<search::AttributeVector *> & getWritableAttributes() const override;
void asyncForEachAttribute(std::shared_ptr<IConstAttributeFunctor> func) const override;
void asyncForEachAttribute(std::shared_ptr<IAttributeFunctor> func, OnDone onDone) const override;
- ExclusiveAttributeReadAccessor::UP getExclusiveReadAccessor(const vespalib::string &name) const override;
void setImportedAttributes(std::unique_ptr<ImportedAttributesRepo> attributes) override;
const ImportedAttributesRepo *getImportedAttributes() const override;
std::shared_ptr<search::attribute::ReadableAttributeVector> readable_attribute_vector(const string& name) const override;
diff --git a/searchcore/src/vespa/searchcore/proton/attribute/i_attribute_manager.h b/searchcore/src/vespa/searchcore/proton/attribute/i_attribute_manager.h
index 437e7bd0208..5e60e950cc4 100644
--- a/searchcore/src/vespa/searchcore/proton/attribute/i_attribute_manager.h
+++ b/searchcore/src/vespa/searchcore/proton/attribute/i_attribute_manager.h
@@ -2,7 +2,6 @@
#pragma once
-#include "exclusive_attribute_read_accessor.h"
#include "i_attribute_factory.h"
#include <vespa/searchcommon/attribute/i_attribute_functor.h>
#include <vespa/searchcore/proton/common/i_transient_resource_usage_provider.h>
@@ -102,8 +101,6 @@ struct IAttributeManager : public search::IAttributeManager
virtual void asyncForEachAttribute(std::shared_ptr<IConstAttributeFunctor> func) const = 0;
virtual void asyncForEachAttribute(std::shared_ptr<IAttributeFunctor> func, OnDone onDone) const = 0;
- virtual ExclusiveAttributeReadAccessor::UP getExclusiveReadAccessor(const vespalib::string &name) const = 0;
-
virtual void setImportedAttributes(std::unique_ptr<ImportedAttributesRepo> attributes) = 0;
virtual const ImportedAttributesRepo *getImportedAttributes() const = 0;
diff --git a/searchcore/src/vespa/searchcore/proton/documentmetastore/document_meta_store_explorer.cpp b/searchcore/src/vespa/searchcore/proton/documentmetastore/document_meta_store_explorer.cpp
index 5a1f0f42507..ffb9acd5501 100644
--- a/searchcore/src/vespa/searchcore/proton/documentmetastore/document_meta_store_explorer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/documentmetastore/document_meta_store_explorer.cpp
@@ -1,8 +1,12 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "document_meta_store_explorer.h"
+#include "documentmetastore.h"
+#include <vespa/searchlib/util/state_explorer_utils.h>
#include <vespa/vespalib/data/slime/cursor.h>
+using search::StateExplorerUtils;
+using search::attribute::Status;
using vespalib::slime::Cursor;
using vespalib::slime::Inserter;
@@ -18,6 +22,11 @@ DocumentMetaStoreExplorer::get_state(const Inserter &inserter, bool full) const
{
Cursor &object = inserter.insertObject();
if (full) {
+ auto dms = dynamic_cast<const DocumentMetaStore*>(&_metaStore->get());
+ if (dms != nullptr) {
+ const Status &status = dms->getStatus();
+ StateExplorerUtils::status_to_slime(status, object.setObject("status"));
+ }
search::LidUsageStats stats = _metaStore->get().getLidUsageStats();
object.setLong("usedLids", stats.getUsedLids());
object.setLong("activeLids", _metaStore->get().getNumActiveLids());
diff --git a/searchcore/src/vespa/searchcore/proton/test/mock_attribute_manager.h b/searchcore/src/vespa/searchcore/proton/test/mock_attribute_manager.h
index f85baa8c0ac..babecb6a77d 100644
--- a/searchcore/src/vespa/searchcore/proton/test/mock_attribute_manager.h
+++ b/searchcore/src/vespa/searchcore/proton/test/mock_attribute_manager.h
@@ -91,9 +91,6 @@ public:
void asyncForEachAttribute(std::shared_ptr<IConstAttributeFunctor>) const override { }
void asyncForEachAttribute(std::shared_ptr<IAttributeFunctor>, OnDone) const override { }
- ExclusiveAttributeReadAccessor::UP getExclusiveReadAccessor(const vespalib::string &) const override {
- return ExclusiveAttributeReadAccessor::UP();
- }
void setImportedAttributes(std::unique_ptr<ImportedAttributesRepo> importedAttributes) override {
_importedAttributes = std::move(importedAttributes);
}
diff --git a/searchlib/src/vespa/searchlib/util/state_explorer_utils.cpp b/searchlib/src/vespa/searchlib/util/state_explorer_utils.cpp
index 00e2e081302..d61737d2a5f 100644
--- a/searchlib/src/vespa/searchlib/util/state_explorer_utils.cpp
+++ b/searchlib/src/vespa/searchlib/util/state_explorer_utils.cpp
@@ -1,13 +1,18 @@
// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "state_explorer_utils.h"
+#include <vespa/searchcommon/attribute/status.h>
#include <vespa/vespalib/data/slime/cursor.h>
#include <vespa/vespalib/util/memoryusage.h>
+using search::attribute::Status;
+using vespalib::MemoryUsage;
+using vespalib::slime::Cursor;
+
namespace search {
void
-StateExplorerUtils::memory_usage_to_slime(const vespalib::MemoryUsage& usage, vespalib::slime::Cursor& object)
+StateExplorerUtils::memory_usage_to_slime(const MemoryUsage& usage, Cursor& object)
{
object.setLong("allocated", usage.allocatedBytes());
object.setLong("used", usage.usedBytes());
@@ -15,5 +20,24 @@ StateExplorerUtils::memory_usage_to_slime(const vespalib::MemoryUsage& usage, ve
object.setLong("onHold", usage.allocatedBytesOnHold());
}
+void
+StateExplorerUtils::status_to_slime(const Status &status, Cursor &object)
+{
+ object.setLong("numDocs", status.getNumDocs());
+ object.setLong("numValues", status.getNumValues());
+ object.setLong("numUniqueValues", status.getNumUniqueValues());
+ object.setLong("lastSerialNum", status.getLastSyncToken());
+ object.setLong("updateCount", status.getUpdateCount());
+ object.setLong("nonIdempotentUpdateCount", status.getNonIdempotentUpdateCount());
+ object.setLong("bitVectors", status.getBitVectors());
+ {
+ Cursor &memory = object.setObject("memoryUsage");
+ memory.setLong("allocatedBytes", status.getAllocated());
+ memory.setLong("usedBytes", status.getUsed());
+ memory.setLong("deadBytes", status.getDead());
+ memory.setLong("onHoldBytes", status.getOnHold());
+ memory.setLong("onHoldBytesMax", status.getOnHoldMax());
+ }
}
+}
diff --git a/searchlib/src/vespa/searchlib/util/state_explorer_utils.h b/searchlib/src/vespa/searchlib/util/state_explorer_utils.h
index eebac033714..9a8d1a7d9db 100644
--- a/searchlib/src/vespa/searchlib/util/state_explorer_utils.h
+++ b/searchlib/src/vespa/searchlib/util/state_explorer_utils.h
@@ -2,6 +2,7 @@
#pragma once
+namespace search::attribute { class Status; }
namespace vespalib { class MemoryUsage; }
namespace vespalib::slime { struct Cursor; }
@@ -13,6 +14,7 @@ namespace search {
class StateExplorerUtils {
public:
static void memory_usage_to_slime(const vespalib::MemoryUsage& usage, vespalib::slime::Cursor& object);
+ static void status_to_slime(const search::attribute::Status &status, vespalib::slime::Cursor &object);
};
}
diff --git a/storage/src/vespa/storage/frameworkimpl/thread/appkiller.h b/storage/src/vespa/storage/frameworkimpl/thread/appkiller.h
index de8005bf914..4c6b25c9d2c 100644
--- a/storage/src/vespa/storage/frameworkimpl/thread/appkiller.h
+++ b/storage/src/vespa/storage/frameworkimpl/thread/appkiller.h
@@ -17,7 +17,7 @@ namespace storage {
struct AppKiller {
using UP = std::unique_ptr<AppKiller>;
- virtual ~AppKiller() {}
+ virtual ~AppKiller() = default;
virtual void kill() = 0;
};
diff --git a/storage/src/vespa/storage/storageserver/servicelayernode.h b/storage/src/vespa/storage/storageserver/servicelayernode.h
index 4f4af71ef57..4b719a0330b 100644
--- a/storage/src/vespa/storage/storageserver/servicelayernode.h
+++ b/storage/src/vespa/storage/storageserver/servicelayernode.h
@@ -44,7 +44,7 @@ public:
ApplicationGenerationFetcher& generationFetcher,
spi::PersistenceProvider& persistenceProvider,
const VisitorFactory::Map& externalVisitors);
- ~ServiceLayerNode();
+ ~ServiceLayerNode() override;
/**
* Init must be called exactly once after construction and before destruction.
*/
diff --git a/storage/src/vespa/storage/storageserver/storagenode.cpp b/storage/src/vespa/storage/storageserver/storagenode.cpp
index 9f8456afc37..f0f410d3076 100644
--- a/storage/src/vespa/storage/storageserver/storagenode.cpp
+++ b/storage/src/vespa/storage/storageserver/storagenode.cpp
@@ -364,7 +364,7 @@ StorageNode::shutdown()
LOG(debug, "Storage killed before requestShutdown() was called. No "
"reason has been given for why we're stopping.");
}
- // Remove the subscription to avoid more callbacks from config
+ // Remove the subscription to avoid more callbacks from config
removeConfigSubscriptions();
if (_chain) {
diff --git a/storage/src/vespa/storage/storageserver/storagenode.h b/storage/src/vespa/storage/storageserver/storagenode.h
index 19b930c184f..344181adad5 100644
--- a/storage/src/vespa/storage/storageserver/storagenode.h
+++ b/storage/src/vespa/storage/storageserver/storagenode.h
@@ -138,18 +138,18 @@ private:
void configure(std::unique_ptr<BucketspacesConfig>) override;
protected:
- // Lock taken while doing configuration of the server.
+ // Lock taken while doing configuration of the server.
std::mutex _configLock;
std::mutex _initial_config_mutex;
using InitialGuard = std::lock_guard<std::mutex>;
- // Current running config. Kept, such that we can see what has been
- // changed in live config updates.
+ // Current running config. Kept, such that we can see what has been
+ // changed in live config updates.
std::unique_ptr<StorServerConfig> _serverConfig;
std::unique_ptr<UpgradingConfig> _clusterConfig;
std::unique_ptr<StorDistributionConfig> _distributionConfig;
std::unique_ptr<document::config::DocumenttypesConfig> _doctypesConfig;
std::unique_ptr<BucketspacesConfig> _bucketSpacesConfig;
- // New configs gotten that has yet to have been handled
+ // New configs gotten that has yet to have been handled
std::unique_ptr<StorServerConfig> _newServerConfig;
std::unique_ptr<UpgradingConfig> _newClusterConfig;
std::unique_ptr<StorDistributionConfig> _newDistributionConfig;
diff --git a/storage/src/vespa/storageframework/generic/thread/thread.h b/storage/src/vespa/storageframework/generic/thread/thread.h
index 6be65667b0f..80ddf33b79a 100644
--- a/storage/src/vespa/storageframework/generic/thread/thread.h
+++ b/storage/src/vespa/storageframework/generic/thread/thread.h
@@ -32,15 +32,15 @@ class Thread : public ThreadHandle {
public:
using UP = std::unique_ptr<Thread>;
- Thread(vespalib::stringref id) : _id(id) {}
- virtual ~Thread() = default;
+ explicit Thread(vespalib::stringref id) : _id(id) {}
+ ~Thread() override = default;
- virtual const vespalib::string& getId() const { return _id; }
+ [[nodiscard]] virtual const vespalib::string& getId() const { return _id; }
/** Check whether thread have been interrupted or not. */
- virtual bool interrupted() const override = 0;
+ [[nodiscard]] bool interrupted() const override = 0;
/** Check whether thread have been joined or not. */
- virtual bool joined() const = 0;
+ [[nodiscard]] virtual bool joined() const = 0;
/**
* Call this function to set interrupt flag, such that later calls to
diff --git a/vdslib/src/main/java/com/yahoo/vdslib/distribution/Distribution.java b/vdslib/src/main/java/com/yahoo/vdslib/distribution/Distribution.java
index 4f9186a0569..9a451ac56ec 100644
--- a/vdslib/src/main/java/com/yahoo/vdslib/distribution/Distribution.java
+++ b/vdslib/src/main/java/com/yahoo/vdslib/distribution/Distribution.java
@@ -27,19 +27,17 @@ import java.util.concurrent.atomic.AtomicReference;
public class Distribution {
private static class Config {
- Config(Group nodeGraph, int redundancy, boolean distributorAutoOwnershipTransferOnWholeGroupDown) {
+ Config(Group nodeGraph, int redundancy) {
this.nodeGraph = nodeGraph;
this.redundancy = redundancy;
- this.distributorAutoOwnershipTransferOnWholeGroupDown = distributorAutoOwnershipTransferOnWholeGroupDown;
}
private final Group nodeGraph;
private final int redundancy;
- private final boolean distributorAutoOwnershipTransferOnWholeGroupDown;
}
private ConfigSubscriber configSub;
- private final AtomicReference<Config> config = new AtomicReference<>(new Config(null, 1, false));
+ private final AtomicReference<Config> config = new AtomicReference<>(new Config(null, 1));
public Group getRootGroup() {
return config.getAcquire().nodeGraph;
@@ -96,7 +94,7 @@ public class Distribution {
if (root == null)
throw new IllegalStateException("Config does not specify a root group");
root.calculateDistributionHashValues();
- Distribution.this.config.setRelease(new Config(root, config.redundancy(), config.distributor_auto_ownership_transfer_on_whole_group_down()));
+ Distribution.this.config.setRelease(new Config(root, config.redundancy()));
} catch (ParseException e) {
throw new IllegalStateException("Failed to parse config", e);
}
@@ -139,7 +137,7 @@ public class Distribution {
if (root == null)
throw new IllegalStateException("Config does not specify a root group");
root.calculateDistributionHashValues();
- Distribution.this.config.setRelease(new Config(root, config.redundancy(), true));
+ Distribution.this.config.setRelease(new Config(root, config.redundancy()));
} catch (ParseException e) {
throw new IllegalStateException("Failed to parse config", e);
}
@@ -241,8 +239,7 @@ public class Distribution {
return true;
}
- private Group getIdealDistributorGroup(boolean distributorAutoOwnershipTransferOnWholeGroupDown,
- BucketId bucket, ClusterState clusterState, Group parent, int redundancy) {
+ private Group getIdealDistributorGroup(BucketId bucket, ClusterState clusterState, Group parent, int redundancy) {
if (parent.isLeafGroup()) {
return parent;
}
@@ -259,15 +256,13 @@ public class Distribution {
}
results.add(new ScoredGroup(g, score));
}
- if (distributorAutoOwnershipTransferOnWholeGroupDown) {
- while (!results.isEmpty() && allDistributorsDown(results.first().group, clusterState)) {
- results.remove(results.first());
- }
+ while (!results.isEmpty() && allDistributorsDown(results.first().group, clusterState)) {
+ results.remove(results.first());
}
if (results.isEmpty()) {
return null;
}
- return getIdealDistributorGroup(distributorAutoOwnershipTransferOnWholeGroupDown, bucket, clusterState, results.first().group, redundancyArray[0]);
+ return getIdealDistributorGroup(bucket, clusterState, results.first().group, redundancyArray[0]);
}
private static class ResultGroup implements Comparable<ResultGroup> {
@@ -434,7 +429,7 @@ public class Distribution {
}
Config cfg = config.getAcquire();
- Group idealGroup = getIdealDistributorGroup(cfg.distributorAutoOwnershipTransferOnWholeGroupDown, bucket, state, cfg.nodeGraph, cfg.redundancy);
+ Group idealGroup = getIdealDistributorGroup(bucket, state, cfg.nodeGraph, cfg.redundancy);
if (idealGroup == null) {
throw new NoDistributorsAvailableException("No distributors available in cluster state version " + state.getVersion());
}
diff --git a/vdslib/src/test/java/com/yahoo/vdslib/distribution/DistributionTestCase.java b/vdslib/src/test/java/com/yahoo/vdslib/distribution/DistributionTestCase.java
index 415c4ffe8f0..19c9c79522d 100644
--- a/vdslib/src/test/java/com/yahoo/vdslib/distribution/DistributionTestCase.java
+++ b/vdslib/src/test/java/com/yahoo/vdslib/distribution/DistributionTestCase.java
@@ -345,36 +345,6 @@ public class DistributionTestCase {
}
@Test
- public void testDistributorNoGroupTakeover() throws Exception {
- test = new DistributionTestFactory("hierarchical-grouping-distributor-notakeover")
- .setDistribution(buildHierarchicalConfig(6, 3, 1, "1|2|*", 3).distributor_auto_ownership_transfer_on_whole_group_down(false))
- .setNodeType(NodeType.DISTRIBUTOR)
- .setClusterState(new ClusterState("distributor:2 storage:9"));
- int [] counts = new int[10];
- int noneExisting = 0;
- for (BucketId bucket : getTestBuckets()) {
- DistributionTestFactory.Test t = test.recordResult(bucket);
- List<Integer> nodes = t.getNodes();
- if (nodes.isEmpty()) {
- ++noneExisting;
- t.assertFailure(DistributionTestFactory.Failure.NO_DISTRIBUTORS_AVAILABLE);
- } else {
- t.assertNodeCount(1);
- for (int i : nodes) {
- ++counts[i];
- }
- }
- }
- for (int i=2; i<10; ++i) {
- assertEquals(0, counts[i]);
- }
- for (int i=0; i<2; ++i) {
- assertTrue(counts[i] > 0);
- }
- assertEquals(15, noneExisting);
- }
-
- @Test
public void testHierarchicalDistributionDeep() throws Exception {
System.out.println(new StorDistributionConfig(buildHierarchicalConfig(8, 5, 3, "*|*", 3)));
test = new DistributionTestFactory("hierarchical-grouping-deep")
diff --git a/vdslib/src/tests/distribution/testdata/hierarchical-grouping-distributor-notakeover.java.results b/vdslib/src/tests/distribution/testdata/hierarchical-grouping-distributor-notakeover.java.results
deleted file mode 100644
index f092339a37a..00000000000
--- a/vdslib/src/tests/distribution/testdata/hierarchical-grouping-distributor-notakeover.java.results
+++ /dev/null
@@ -1,585 +0,0 @@
-{
- "cluster-state" : "distributor:2 storage:9",
- "distribution" : "redundancy 6\ninitial_redundancy 0\nensure_primary_persisted true\nready_copies 0\nactive_per_leaf_group false\ndistributor_auto_ownership_transfer_on_whole_group_down false\ngroup[0].index \"invalid\"\ngroup[0].name \"invalid\"\ngroup[0].capacity 1.0\ngroup[0].partitions \"1|2|*\"\ngroup[1].index \"0\"\ngroup[1].name \"top.0\"\ngroup[1].capacity 1.0\ngroup[1].partitions \"\"\ngroup[1].nodes[0].index 8\ngroup[1].nodes[0].retired false\ngroup[1].nodes[1].index 1\ngroup[1].nodes[1].retired false\ngroup[1].nodes[2].index 6\ngroup[1].nodes[2].retired false\ngroup[2].index \"1\"\ngroup[2].name \"top.1\"\ngroup[2].capacity 1.0\ngroup[2].partitions \"\"\ngroup[2].nodes[0].index 5\ngroup[2].nodes[0].retired false\ngroup[2].nodes[1].index 0\ngroup[2].nodes[1].retired false\ngroup[2].nodes[2].index 2\ngroup[2].nodes[2].retired false\ngroup[3].index \"2\"\ngroup[3].name \"top.2\"\ngroup[3].capacity 1.0\ngroup[3].partitions \"\"\ngroup[3].nodes[0].index 4\ngroup[3].nodes[0].retired false\ngroup[3].nodes[1].index 3\ngroup[3].nodes[1].retired false\ngroup[3].nodes[2].index 7\ngroup[3].nodes[2].retired false\ndisk_distribution MODULO_BID",
- "node-type" : "distributor",
- "redundancy" : 3,
- "node-count" : 10,
- "up-states" : "uim",
- "result" : [ {
- "nodes" : [ 0 ],
- "bucket" : "4000000000000000",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4000000000000001",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4000000000000002",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "4000000000000003",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4000000000000004",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4000000000000005",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4000000000000006",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4000000000000007",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4000000000000008",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4000000000000009",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "400000000000000a",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "400000000000000b",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "400000000000000c",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ ],
- "bucket" : "400000000000000d",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "400000000000000e",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "400000000000000f",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4000000000000010",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "4000000000000011",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4000000000000012",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4000000000000013",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4400000000000000",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4400000000000001",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4400000000000002",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "4400000000000003",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4400000000000004",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4400000000000005",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4400000000000006",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4400000000000007",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4400000000000008",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4400000000000009",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "440000000000000a",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "440000000000000b",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "440000000000000c",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ ],
- "bucket" : "440000000000000d",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "440000000000000e",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "440000000000000f",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4400000000000010",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "4400000000000011",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4400000000000012",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4400000000000013",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4800000000000000",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4800000000000001",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4800000000000002",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "4800000000000003",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4800000000000004",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4800000000000005",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4800000000000006",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4800000000000007",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4800000000000008",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4800000000000009",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "480000000000000a",
- "failure" : "NONE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "480000000000000b",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "480000000000000c",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ ],
- "bucket" : "480000000000000d",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "480000000000000e",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "480000000000000f",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4800000000000010",
- "failure" : "NONE"
- }, {
- "nodes" : [ ],
- "bucket" : "4800000000000011",
- "failure" : "NO_DISTRIBUTORS_AVAILABLE"
- }, {
- "nodes" : [ 0 ],
- "bucket" : "4800000000000012",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4800000000000013",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "400000000000fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "440000000001fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "480000000001fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4c0000000001fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "500000000009fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "540000000009fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "580000000009fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "5c0000000049fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "600000000049fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "640000000149fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "680000000349fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "6c0000000749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "700000000749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "740000001749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "780000003749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "7c0000003749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "80000000b749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "84000001b749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "88000003b749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "8c000007b749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "9000000fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "9400001fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "9800003fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "9c00007fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "a000007fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "a400007fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "a800007fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "ac00047fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "b000047fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "b400147fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "b800347fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "bc00347fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "c000b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "c400b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "c800b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "cc00b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "d000b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "d400b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "d800b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "dc00b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "e000b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "e400b47fb749fe68",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "40000000000098d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "44000000000098d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "48000000000298d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "4c000000000698d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "50000000000e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "54000000000e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "58000000002e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "5c000000002e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "60000000002e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "64000000012e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "68000000012e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "6c000000052e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "70000000052e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "74000000152e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "78000000152e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "7c000000152e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "80000000952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "84000001952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "88000001952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "8c000001952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "90000001952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "94000001952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "98000001952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "9c000041952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "a00000c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "a40000c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "a80002c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "ac0002c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "b00002c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "b40012c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "b80012c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "bc0012c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "c00092c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "c40192c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "c80192c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "cc0192c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "d00992c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "d41992c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "d83992c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "dc7992c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "e07992c1952e98d2",
- "failure" : "NONE"
- }, {
- "nodes" : [ 1 ],
- "bucket" : "e47992c1952e98d2",
- "failure" : "NONE"
- } ]
-} \ No newline at end of file
diff --git a/vespa-dependencies-enforcer/allowed-maven-dependencies.txt b/vespa-dependencies-enforcer/allowed-maven-dependencies.txt
index 18475a8e4d9..95f1179b1a3 100644
--- a/vespa-dependencies-enforcer/allowed-maven-dependencies.txt
+++ b/vespa-dependencies-enforcer/allowed-maven-dependencies.txt
@@ -80,7 +80,7 @@ net.java.dev.jna:jna:5.11.0
net.openhft:zero-allocation-hashing:0.16
org.antlr:antlr-runtime:3.5.3
org.antlr:antlr4-runtime:4.11.1
-org.apache.aries.spifly:org.apache.aries.spifly.dynamic.bundle:1.3.5
+org.apache.aries.spifly:org.apache.aries.spifly.dynamic.bundle:1.3.6
org.apache.commons:commons-compress:1.22
org.apache.commons:commons-csv:1.8
org.apache.commons:commons-exec:1.3
diff --git a/vespajlib/src/main/java/com/yahoo/slime/BinaryDecoder.java b/vespajlib/src/main/java/com/yahoo/slime/BinaryDecoder.java
index b9aa8e5cf22..af6a2ae80a3 100644
--- a/vespajlib/src/main/java/com/yahoo/slime/BinaryDecoder.java
+++ b/vespajlib/src/main/java/com/yahoo/slime/BinaryDecoder.java
@@ -23,7 +23,7 @@ final class BinaryDecoder {
public Slime decode(byte[] bytes, int offset, int length) {
Slime slime = new Slime();
in = new BufferedInput(bytes, offset, length);
- decodeSymbolTable(slime);
+ decodeSymbolTable(in, slime.symbolTable());
decodeValue(slimeInserter.adjust(slime));
if (in.failed()) {
slime.wrap("partial_result");
@@ -33,22 +33,6 @@ final class BinaryDecoder {
return slime;
}
- long read_cmpr_long() {
- long next = in.getByte();
- long value = (next & 0x7f);
- int shift = 7;
- while ((next & 0x80) != 0) {
- next = in.getByte();
- value |= ((next & 0x7f) << shift);
- shift += 7;
- }
- return value;
- }
-
- long read_size(int meta) {
- return (meta == 0) ? read_cmpr_long() : (meta - 1);
- }
-
long read_bytes_le(int bytes) {
long value = 0;
int shift = 0;
@@ -90,22 +74,20 @@ final class BinaryDecoder {
}
Cursor decodeSTRING(Inserter inserter, int meta) {
- long size = read_size(meta);
- int sz = (int)size; // XXX
- byte[] image = in.getBytes(sz);
+ int size = in.read_size(meta);
+ byte[] image = in.getBytes(size);
return inserter.insertSTRING(image);
}
Cursor decodeDATA(Inserter inserter, int meta) {
- long size = read_size(meta);
- int sz = (int)size; // XXX
- byte[] image = in.getBytes(sz);
+ int size = in.read_size(meta);
+ byte[] image = in.getBytes(size);
return inserter.insertDATA(image);
}
Cursor decodeARRAY(Inserter inserter, int meta) {
Cursor cursor = inserter.insertARRAY();
- long size = read_size(meta);
+ int size = in.read_size(meta);
for (int i = 0; i < size; ++i) {
decodeValue(arrayInserter.adjust(cursor));
}
@@ -114,10 +96,9 @@ final class BinaryDecoder {
Cursor decodeOBJECT(Inserter inserter, int meta) {
Cursor cursor = inserter.insertOBJECT();
- long size = read_size(meta);
+ int size = in.read_size(meta);
for (int i = 0; i < size; ++i) {
- long l = read_cmpr_long();
- int symbol = (int)l; // check for overflow?
+ int symbol = in.read_cmpr_int();
decodeValue(objectInserter.adjust(cursor, symbol));
}
return cursor;
@@ -146,20 +127,18 @@ final class BinaryDecoder {
}
}
- void decodeSymbolTable(Slime slime) {
- long numSymbols = read_cmpr_long();
- final byte [] backing = in.getBacking();
+ static void decodeSymbolTable(BufferedInput input, SymbolTable names) {
+ int numSymbols = input.read_cmpr_int();
+ final byte[] backing = input.getBacking();
for (int i = 0; i < numSymbols; ++i) {
- long size = read_cmpr_long();
- int sz = (int)size; // XXX
- int offset = in.getPosition();
- in.skip(sz);
- int symbol = slime.insert(Utf8Codec.decode(backing, offset, sz));
+ int size = input.read_cmpr_int();
+ int offset = input.getPosition();
+ input.skip(size);
+ int symbol = names.insert(Utf8Codec.decode(backing, offset, size));
if (symbol != i) {
- in.fail("duplicate symbols in symbol table");
+ input.fail("duplicate symbols in symbol table");
return;
}
}
}
-
}
diff --git a/vespajlib/src/main/java/com/yahoo/slime/BinaryEncoder.java b/vespajlib/src/main/java/com/yahoo/slime/BinaryEncoder.java
index 7da85b5cb63..f12496f7a76 100644
--- a/vespajlib/src/main/java/com/yahoo/slime/BinaryEncoder.java
+++ b/vespajlib/src/main/java/com/yahoo/slime/BinaryEncoder.java
@@ -24,7 +24,7 @@ final class BinaryEncoder implements ArrayTraverser, ObjectSymbolTraverser {
}
- void encode_cmpr_long(long value) {
+ void encode_cmpr_int(int value) {
byte next = (byte)(value & 0x7f);
value >>>= 7; // unsigned shift
while (value != 0) {
@@ -36,12 +36,12 @@ final class BinaryEncoder implements ArrayTraverser, ObjectSymbolTraverser {
out.put(next);
}
- void write_type_and_size(int type, long size) {
+ void write_type_and_size(int type, int size) {
if (size <= 30) {
- out.put(encode_type_and_meta(type, (int)(size + 1)));
+ out.put(encode_type_and_meta(type, size + 1));
} else {
out.put(encode_type_and_meta(type, 0));
- encode_cmpr_long(size);
+ encode_cmpr_int(size);
}
}
@@ -125,11 +125,11 @@ final class BinaryEncoder implements ArrayTraverser, ObjectSymbolTraverser {
void encodeSymbolTable(Slime slime) {
int numSymbols = slime.symbols();
- encode_cmpr_long(numSymbols);
+ encode_cmpr_int(numSymbols);
for (int i = 0 ; i < numSymbols; ++i) {
String name = slime.inspect(i);
byte[] bytes = Utf8Codec.encode(name);
- encode_cmpr_long(bytes.length);
+ encode_cmpr_int(bytes.length);
out.put(bytes);
}
}
@@ -139,7 +139,7 @@ final class BinaryEncoder implements ArrayTraverser, ObjectSymbolTraverser {
}
public void field(int symbol, Inspector inspector) {
- encode_cmpr_long(symbol);
+ encode_cmpr_int(symbol);
encodeValue(inspector);
}
diff --git a/vespajlib/src/main/java/com/yahoo/slime/BinaryView.java b/vespajlib/src/main/java/com/yahoo/slime/BinaryView.java
new file mode 100644
index 00000000000..0e111d42061
--- /dev/null
+++ b/vespajlib/src/main/java/com/yahoo/slime/BinaryView.java
@@ -0,0 +1,307 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.slime;
+
+import java.util.function.Consumer;
+import static com.yahoo.slime.BinaryFormat.decode_double;
+import static com.yahoo.slime.BinaryFormat.decode_meta;
+import static com.yahoo.slime.BinaryFormat.decode_type;
+import static com.yahoo.slime.BinaryFormat.decode_zigzag;
+
+/**
+ * A read-only view of a Slime value that is stored in binary format.
+ **/
+public final class BinaryView implements Inspector {
+
+ private final byte[] data;
+ private final SymbolTable names;
+ private final DecodeIndex index;
+ private final int self;
+
+ private BinaryView(byte[] data, SymbolTable names, DecodeIndex index, int self) {
+ this.data = data;
+ this.names = names;
+ this.index = index;
+ this.self = self;
+ }
+ private int peek_cmpr_int(int idx) {
+ long next = data[idx++];
+ long value = (next & 0x7f);
+ int shift = 7;
+ while ((next & 0x80) != 0) {
+ next = data[idx++];
+ value |= ((next & 0x7f) << shift);
+ shift += 7;
+ }
+ return (int)value;
+ }
+ private int skip_cmpr_int(int idx) {
+ while ((data[idx++] & 0x80) != 0);
+ return idx;
+ }
+ private int extract_children(int idx) {
+ int bytes = decode_meta(data[idx++]);
+ return (bytes == 0)
+ ? peek_cmpr_int(idx)
+ : (bytes - 1);
+ }
+ private long extract_long(int idx) {
+ int bytes = decode_meta(data[idx++]);
+ long value = 0;
+ int shift = 0;
+ for (int i = 0; i < bytes; ++i) {
+ long b = data[idx++];
+ value |= (b & 0xff) << shift;
+ shift += 8;
+ }
+ return decode_zigzag(value);
+ }
+ private double extract_double(int idx) {
+ int bytes = decode_meta(data[idx++]);
+ long value = 0;
+ int shift = 56;
+ for (int i = 0; i < bytes; ++i) {
+ long b = data[idx++];
+ value |= (b & 0xff) << shift;
+ shift -= 8;
+ }
+ return decode_double(value);
+ }
+ private String extract_string(int idx) {
+ int bytes = decode_meta(data[idx++]);
+ if (bytes == 0) {
+ bytes = peek_cmpr_int(idx);
+ idx = skip_cmpr_int(idx);
+ } else {
+ --bytes;
+ }
+ return Utf8Codec.decode(data, idx, bytes);
+ }
+ private byte[] extract_bytes(int idx) {
+ int bytes = decode_meta(data[idx++]);
+ if (bytes == 0) {
+ bytes = peek_cmpr_int(idx);
+ idx = skip_cmpr_int(idx);
+ } else {
+ --bytes;
+ }
+ byte[] ret = new byte[bytes];
+ for (int i = 0; i < bytes; ++i) {
+ ret[i] = data[idx++];
+ }
+ return ret;
+ }
+ private Inspector find_field(int pos, int len, int sym) {
+ for (int i = 0; i < len; ++i) {
+ int idx = index.getByteOffset(pos + i);
+ if (peek_cmpr_int(idx - (index.getExtBits(pos + i) + 1)) == sym) {
+ return new BinaryView(data, names, index, pos + i);
+ }
+ }
+ return NixValue.invalid();
+ }
+
+ @Override public boolean valid() { return true; }
+ @Override public void ifValid(Consumer<Inspector> consumer) { consumer.accept(this); }
+ @Override public Type type() { return decode_type(data[index.getByteOffset(self)]); }
+ @Override public int children() {
+ return switch (type()) {
+ case OBJECT, ARRAY -> extract_children(index.getByteOffset(self));
+ default -> 0;
+ };
+ }
+ @Override public int entries() {
+ return switch (type()) {
+ case ARRAY -> extract_children(index.getByteOffset(self));
+ default -> 0;
+ };
+ }
+ @Override public int fields() {
+ return switch (type()) {
+ case OBJECT -> extract_children(index.getByteOffset(self));
+ default -> 0;
+ };
+ }
+ @Override public boolean asBool() {
+ return switch (type()) {
+ case BOOL -> (decode_meta(data[index.getByteOffset(self)]) != 0);
+ default -> false;
+ };
+ }
+ @Override public long asLong() {
+ return switch (type()) {
+ case LONG -> extract_long(index.getByteOffset(self));
+ case DOUBLE -> (long)extract_double(index.getByteOffset(self));
+ default -> 0;
+ };
+ }
+ @Override public double asDouble() {
+ return switch (type()) {
+ case LONG -> extract_long(index.getByteOffset(self));
+ case DOUBLE -> extract_double(index.getByteOffset(self));
+ default -> 0.0;
+ };
+ }
+ @Override public String asString() {
+ return switch (type()) {
+ case STRING -> extract_string(index.getByteOffset(self));
+ default -> Value.emptyString;
+ };
+ }
+ @Override public byte[] asUtf8() {
+ return switch (type()) {
+ case STRING -> extract_bytes(index.getByteOffset(self));
+ default -> Value.emptyData;
+ };
+ }
+ @Override public byte[] asData() {
+ return switch (type()) {
+ case DATA -> extract_bytes(index.getByteOffset(self));
+ default -> Value.emptyData;
+ };
+ }
+ @Override public void accept(Visitor v) {
+ switch (type()) {
+ case NIX: v.visitNix(); break;
+ case BOOL: v.visitBool(decode_meta(data[index.getByteOffset(self)]) != 0); break;
+ case LONG: v.visitLong(extract_long(index.getByteOffset(self))); break;
+ case DOUBLE: v.visitDouble(extract_double(index.getByteOffset(self))); break;
+ case STRING: v.visitString(extract_bytes(index.getByteOffset(self))); break;
+ case DATA: v.visitData(extract_bytes(index.getByteOffset(self))); break;
+ case ARRAY: v.visitArray(this); break;
+ case OBJECT: v.visitObject(this); break;
+ default: throw new RuntimeException("should not be reached");
+ }
+ }
+ @Override public void traverse(ArrayTraverser at) {
+ int pos = index.getFirstChild(self);
+ int len = entries();
+ for (int i = 0; i < len; ++i) {
+ at.entry(i, new BinaryView(data, names, index, pos + i));
+ }
+ }
+ @Override public void traverse(ObjectSymbolTraverser ot) {
+ int pos = index.getFirstChild(self);
+ int len = fields();
+ for (int i = 0; i < len; ++i) {
+ int sym = peek_cmpr_int(index.getByteOffset(pos + i) - (index.getExtBits(pos + i) + 1));
+ ot.field(sym, new BinaryView(data, names, index, pos + i));
+ }
+ }
+ @Override public void traverse(ObjectTraverser ot) {
+ int pos = index.getFirstChild(self);
+ int len = fields();
+ for (int i = 0; i < len; ++i) {
+ int sym = peek_cmpr_int(index.getByteOffset(pos + i) - (index.getExtBits(pos + i) + 1));
+ ot.field(names.inspect(sym), new BinaryView(data, names, index, pos + i));
+ }
+ }
+ @Override public Inspector entry(int idx) {
+ int limit = entries();
+ if (idx >= 0 && idx < limit) {
+ return new BinaryView(data, names, index, index.getFirstChild(self) + idx);
+ }
+ return NixValue.invalid();
+ }
+ @Override public Inspector field(int sym) {
+ int limit = fields();
+ if (limit > 0 && sym != SymbolTable.INVALID) {
+ return find_field(index.getFirstChild(self), limit, sym);
+ }
+ return NixValue.invalid();
+ }
+ @Override public Inspector field(String name) {
+ int limit = fields();
+ if (limit > 0) {
+ int sym = names.lookup(name);
+ if (sym != SymbolTable.INVALID) {
+ return find_field(index.getFirstChild(self), limit, sym);
+ }
+ }
+ return NixValue.invalid();
+ }
+
+ private static void buildIndex(BufferedInput input, DecodeIndex index, int self, int extBits) {
+ int pos = input.getPosition();
+ byte tag = input.getByte();
+ Type type = decode_type(tag);
+ int meta = decode_meta(tag);
+ switch (type) {
+ case NIX:
+ case BOOL:
+ index.set(self, pos, 0, extBits);
+ break;
+ case LONG:
+ case DOUBLE:
+ input.skip(meta);
+ index.set(self, pos, 0, extBits);
+ break;
+ case STRING:
+ case DATA: {
+ int size = input.read_size(meta);
+ input.skip(size);
+ index.set(self, pos, 0, extBits);
+ break; }
+ case ARRAY: {
+ int size = input.read_size(meta);
+ if (size > input.getBacking().length - index.size()) {
+ input.fail("decode index too big");
+ return;
+ }
+ int firstChild = index.reserve(size);
+ index.set(self, pos, firstChild, extBits);
+ for (int i = 0; i < size; ++i) {
+ buildIndex(input, index, firstChild + i, 0);
+ }
+ break; }
+ case OBJECT: {
+ int size = input.read_size(meta);
+ if (size > input.getBacking().length - index.size()) {
+ input.fail("decode index too big");
+ return;
+ }
+ int firstChild = index.reserve(size);
+ index.set(self, pos, firstChild, extBits);
+ for (int i = 0; i < size; ++i) {
+ int childExtBits = input.skip_cmpr_int();
+ if (childExtBits > 3) {
+ input.fail("symbol id too big");
+ return;
+ }
+ buildIndex(input, index, firstChild + i, childExtBits);
+ }
+ break; }
+ default: throw new RuntimeException("should not be reached");
+ }
+ }
+
+ static Inspector inspectImpl(BufferedInput input) {
+ var names = new SymbolTable();
+ var index = new DecodeIndex();
+ BinaryDecoder.decodeSymbolTable(input, names);
+ buildIndex(input, index, index.reserve(1), 0);
+ if (input.failed()) {
+ return NixValue.invalid();
+ }
+ return new BinaryView(input.getBacking(), names, index, 0);
+ }
+
+ public static Inspector inspect(byte[] data) {
+ return inspectImpl(new BufferedInput(data));
+ }
+
+ static int peek_cmpr_int_for_testing(byte[] data, int idx) {
+ return new BinaryView(data, null, null, -1).peek_cmpr_int(idx);
+ }
+ static int skip_cmpr_int_for_testing(byte[] data, int idx) {
+ return new BinaryView(data, null, null, -1).skip_cmpr_int(idx);
+ }
+ static int extract_children_for_testing(byte[] data, int idx) {
+ return new BinaryView(data, null, null, -1).extract_children(idx);
+ }
+ static long extract_long_for_testing(byte[] data, int idx) {
+ return new BinaryView(data, null, null, -1).extract_long(idx);
+ }
+ static double extract_double_for_testing(byte[] data, int idx) {
+ return new BinaryView(data, null, null, -1).extract_double(idx);
+ }
+}
diff --git a/vespajlib/src/main/java/com/yahoo/slime/BufferedInput.java b/vespajlib/src/main/java/com/yahoo/slime/BufferedInput.java
index ddbb25196b5..5c994d7f793 100644
--- a/vespajlib/src/main/java/com/yahoo/slime/BufferedInput.java
+++ b/vespajlib/src/main/java/com/yahoo/slime/BufferedInput.java
@@ -59,7 +59,7 @@ final class BufferedInput {
return ret;
}
- byte [] getBacking() { return source; }
+ byte[] getBacking() { return source; }
int getPosition() { return position; }
void skip(int size) {
if (position + size > end) {
@@ -80,4 +80,32 @@ final class BufferedInput {
}
return ret;
}
+
+ int read_cmpr_int() {
+ long next = getByte();
+ long value = (next & 0x7f);
+ int shift = 7;
+ while (shift < 32 && (next & 0x80) != 0) {
+ next = getByte();
+ value |= ((next & 0x7f) << shift);
+ shift += 7;
+ }
+ if (value > 0x7fff_ffffL) {
+ fail("compressed int overflow");
+ value = 0;
+ }
+ return (int)value;
+ }
+
+ int skip_cmpr_int() {
+ int extBits = 0;
+ while ((getByte() & 0x80) != 0) {
+ ++extBits;
+ }
+ return extBits;
+ }
+
+ int read_size(int meta) {
+ return (meta == 0) ? read_cmpr_int() : (meta - 1);
+ }
}
diff --git a/vespajlib/src/main/java/com/yahoo/slime/Slime.java b/vespajlib/src/main/java/com/yahoo/slime/Slime.java
index eba9226c8ef..7d29131cbdb 100644
--- a/vespajlib/src/main/java/com/yahoo/slime/Slime.java
+++ b/vespajlib/src/main/java/com/yahoo/slime/Slime.java
@@ -13,6 +13,8 @@ public final class Slime {
private final SymbolTable names = new SymbolTable();
private Value root = NixValue.instance();
+ SymbolTable symbolTable() { return names; }
+
/**
* Construct an empty Slime with an empty top-level value.
*/
diff --git a/vespajlib/src/main/java/com/yahoo/slime/Value.java b/vespajlib/src/main/java/com/yahoo/slime/Value.java
index 1943e77663f..6a1d7b2dd8e 100644
--- a/vespajlib/src/main/java/com/yahoo/slime/Value.java
+++ b/vespajlib/src/main/java/com/yahoo/slime/Value.java
@@ -13,8 +13,8 @@ import java.util.function.Consumer;
*/
abstract class Value implements Cursor {
- private static final String emptyString = "";
- private static final byte[] emptyData = new byte[0];
+ static final String emptyString = "";
+ static final byte[] emptyData = new byte[0];
public final boolean valid() { return this != NixValue.invalid(); }
diff --git a/vespajlib/src/test/java/com/yahoo/slime/BinaryFormatTestCase.java b/vespajlib/src/test/java/com/yahoo/slime/BinaryFormatTestCase.java
index 5c3126ce3cf..db001a9276b 100644
--- a/vespajlib/src/test/java/com/yahoo/slime/BinaryFormatTestCase.java
+++ b/vespajlib/src/test/java/com/yahoo/slime/BinaryFormatTestCase.java
@@ -31,22 +31,51 @@ public class BinaryFormatTestCase {
return encode_type_and_meta(t.ID, meta);
}
- void verify_cmpr_long(long value, byte[] expect) {
+ void verify_cmpr_int(int value, byte[] expect) {
BufferedOutput output = new BufferedOutput();
BinaryEncoder bof = new BinaryEncoder(output);
- bof.encode_cmpr_long(value);
+ bof.encode_cmpr_int(value);
byte[] actual = output.toArray();
assertThat(actual, is(expect));
BinaryDecoder bif = new BinaryDecoder();
bif.in = new BufferedInput(expect);
- long got = bif.read_cmpr_long();
+ int got = bif.in.read_cmpr_int();
assertThat(got, is(value));
+ assertThat(bif.in.failed(), is(false));
+
+ bif = new BinaryDecoder();
+ bif.in = new BufferedInput(expect);
+ got = bif.in.skip_cmpr_int();
+ assertThat(got, is(expect.length - 1));
+ assertThat(bif.in.getPosition(), is(expect.length));
+ assertThat(bif.in.failed(), is(false));
+
+ assertThat(BinaryView.peek_cmpr_int_for_testing(expect, 0), is(value));
+ assertThat(BinaryView.skip_cmpr_int_for_testing(expect, 0), is(expect.length));
+ }
+
+ void verify_read_cmpr_int_fails(byte[] data) {
+ BinaryDecoder bif = new BinaryDecoder();
+ bif.in = new BufferedInput(data);
+ int got = bif.in.read_cmpr_int();
+ assertThat(got, is(0));
+ assertThat(bif.in.failed(), is(true));
+
+ bif = new BinaryDecoder();
+ bif.in = new BufferedInput(data);
+ got = bif.in.skip_cmpr_int();
+ assertThat(got, is(data.length - 1));
+ assertThat(bif.in.getPosition(), is(data.length));
+ assertThat(bif.in.failed(), is(false));
+
+ assertThat(BinaryView.skip_cmpr_int_for_testing(data, 0), is(data.length));
}
// was verifyBasic
void verifyEncoding(Slime slime, byte[] expect) {
assertThat(BinaryFormat.encode(slime), is(expect));
+ assertThat(slime.get().equalTo(BinaryView.inspect(expect)), is(true));
Compressor compressor = new Compressor(CompressionType.LZ4, 3, 2, 0);
Compressor.Compression result = BinaryFormat.encode_and_compress(slime, compressor);
byte [] decompressed = compressor.decompress(result);
@@ -67,7 +96,7 @@ public class BinaryFormatTestCase {
@Test
public void testZigZagConversion() {
- assertThat(encode_zigzag(0), is((long)0));
+ assertThat(encode_zigzag(0), is(0L));
assertThat(decode_zigzag(encode_zigzag(0)), is(0L));
assertThat(encode_zigzag(-1), is(1L));
@@ -134,87 +163,59 @@ public class BinaryFormatTestCase {
}
@Test
- public void testCompressedLong() {
+ public void testCompressedInt() {
{
- long value = 0;
+ int value = 0;
byte[] wanted = { 0 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 127;
+ int value = 127;
byte[] wanted = { 127 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 128;
+ int value = 128;
byte[] wanted = { -128, 1 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 16383;
+ int value = 16383;
byte[] wanted = { -1, 127 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 16384;
+ int value = 16384;
byte[] wanted = { -128, -128, 1 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 2097151;
+ int value = 2097151;
byte[] wanted = { -1, -1, 127 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 2097152;
+ int value = 2097152;
byte[] wanted = { -128, -128, -128, 1 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 268435455;
+ int value = 268435455;
byte[] wanted = { -1, -1, -1, 127 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 268435456;
+ int value = 268435456;
byte[] wanted = { -128, -128, -128, -128, 1 };
- verify_cmpr_long(value, wanted);
- }{
- long value = 34359738367L;
- byte[] wanted = { -1, -1, -1, -1, 127 };
- verify_cmpr_long(value, wanted);
- }{
- long value = 34359738368L;
- byte[] wanted = { -128, -128, -128, -128, -128, 1 };
- verify_cmpr_long(value, wanted);
- }{
- long value = 4398046511103L;
- byte[] wanted = { -1, -1, -1, -1, -1, 127 };
- verify_cmpr_long(value, wanted);
+ verify_cmpr_int(value, wanted);
}{
- long value = 4398046511104L;
- byte[] wanted = { -128, -128, -128, -128, -128, -128, 1 };
- verify_cmpr_long(value, wanted);
+ int value = 0x7fff_ffff;
+ byte[] wanted = { -1, -1, -1, -1, 7 };
+ verify_cmpr_int(value, wanted);
}{
- long value = 562949953421311L;
- byte[] wanted = { -1, -1, -1, -1, -1, -1, 127 };
- verify_cmpr_long(value, wanted);
+ byte[] data = { -1, -1, -1, -1, 8 };
+ verify_read_cmpr_int_fails(data);
}{
- long value = 562949953421312L;
- byte[] wanted = { -128, -128, -128, -128, -128, -128, -128, 1 };
- verify_cmpr_long(value, wanted);
+ byte[] data = { -1, -1, -1, -1, -1, -1, 1 };
+ verify_read_cmpr_int_fails(data);
}{
- long value = 72057594037927935L;
- byte[] wanted = { -1, -1, -1, -1, -1, -1, -1, 127 };
- verify_cmpr_long(value, wanted);
+ byte[] data = { -1, -1, -1, -1, -1, -1, -1, -1, 1 };
+ verify_read_cmpr_int_fails(data);
}{
- long value = 72057594037927936L;
- byte[] wanted = { -128, -128, -128, -128, -128, -128, -128, -128, 1 };
- verify_cmpr_long(value, wanted);
- }{
- long value = 9223372036854775807L;
- byte[] wanted = { -1, -1, -1, -1, -1, -1, -1, -1, 127 };
- verify_cmpr_long(value, wanted);
- }{
- long value = -9223372036854775808L;
- byte[] wanted = { -128, -128, -128, -128, -128, -128, -128, -128, -128, 1 };
- verify_cmpr_long(value, wanted);
- }{
- long value = -1;
- byte[] wanted = { -1, -1, -1, -1, -1, -1, -1, -1, -1, 1 };
- verify_cmpr_long(value, wanted);
+ byte[] data = { -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 1 };
+ verify_read_cmpr_int_fails(data);
}
}
@@ -225,16 +226,16 @@ public class BinaryFormatTestCase {
@Test
public void testTypeAndSizeConversion() {
for (byte type = 0; type < TYPE_LIMIT; ++type) {
- for (long size = 0; size < 500; ++size) {
+ for (int size = 0; size < 500; ++size) {
BufferedOutput expect = new BufferedOutput();
BufferedOutput actual = new BufferedOutput();
if ((size + 1) < META_LIMIT) {
- expect.put(encode_type_and_meta((int)type, (int)(size +1)));
+ expect.put(encode_type_and_meta(type, size +1));
} else {
expect.put(type);
BinaryEncoder encoder = new BinaryEncoder(expect);
- encoder.encode_cmpr_long(size);
+ encoder.encode_cmpr_int(size);
}
{
BinaryEncoder encoder = new BinaryEncoder(actual);
@@ -247,11 +248,13 @@ public class BinaryFormatTestCase {
bif.in = new BufferedInput(got);
byte b = bif.in.getByte();
Type decodedType = decode_type(b);
- long decodedSize = bif.read_size(decode_meta(b));
+ int decodedSize = bif.in.read_size(decode_meta(b));
assertThat(decodedType.ID, is(type));
assertThat(decodedSize, is(size));
assertThat(bif.in.getConsumedSize(), is(got.length));
assertThat(bif.in.failed(), is(false));
+
+ assertThat(BinaryView.extract_children_for_testing(got, 0), is(size));
}
}
@@ -299,6 +302,12 @@ public class BinaryFormatTestCase {
assertThat(decodedBits, is(bits));
assertThat(bif.in.getConsumedSize(), is(expect.length));
assertThat(bif.in.failed(), is(false));
+
+ if (hi != 0) {
+ assertThat(encode_double(BinaryView.extract_double_for_testing(expect, 0)), is(bits));
+ } else {
+ assertThat(encode_zigzag(BinaryView.extract_long_for_testing(expect, 0)), is(bits));
+ }
}
}
}
diff --git a/vespajlib/src/test/java/com/yahoo/slime/BinaryViewTest.java b/vespajlib/src/test/java/com/yahoo/slime/BinaryViewTest.java
new file mode 100644
index 00000000000..568124369d4
--- /dev/null
+++ b/vespajlib/src/test/java/com/yahoo/slime/BinaryViewTest.java
@@ -0,0 +1,418 @@
+// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package com.yahoo.slime;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.function.Consumer;
+
+import org.junit.Test;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+import static com.yahoo.slime.BinaryFormat.encode_type_and_meta;
+
+public class BinaryViewTest {
+ static String makeString(int size) {
+ var str = new StringBuilder();
+ for (int i = 0; i < size; ++i) {
+ str.append("A");
+ }
+ return str.toString();
+ }
+ static byte[] makeData(int size) {
+ byte[] data = new byte[size];
+ for (int i = 0; i < size; ++i) {
+ data[i] = 65;
+ }
+ return data;
+ }
+ static final int numLeafs = 22;
+ static Cursor insertLeaf(Inserter dst, int id) {
+ return switch (id) {
+ case 0 -> dst.insertNIX();
+ case 1 -> dst.insertBOOL(false);
+ case 2 -> dst.insertBOOL(true);
+ case 3 -> dst.insertLONG(42L);
+ case 4 -> dst.insertLONG(-42L);
+ case 5 -> dst.insertLONG(0x1234_5678_8765_4321L);
+ case 6 -> dst.insertLONG(-0x1234_5678_8765_4321L);
+ case 7 -> dst.insertDOUBLE(3.5);
+ case 8 -> dst.insertDOUBLE(1.0/3.0);
+ case 9 -> dst.insertDOUBLE(-3.5);
+ case 10 -> dst.insertDOUBLE(-(1.0/3.0));
+ case 11 -> dst.insertSTRING(makeString(5));
+ case 12 -> dst.insertSTRING(makeString(50));
+ case 13 -> dst.insertSTRING(makeString(300));
+ case 14 -> dst.insertSTRING(makeData(5));
+ case 15 -> dst.insertSTRING(makeData(50));
+ case 16 -> dst.insertSTRING(makeData(300));
+ case 17 -> dst.insertDATA(makeData(5));
+ case 18 -> dst.insertDATA(makeData(50));
+ case 19 -> dst.insertDATA(makeData(300));
+ case 20 -> dst.insertARRAY();
+ case 21 -> dst.insertOBJECT();
+ default -> NixValue.invalid();
+ };
+ }
+ static Cursor insertInnerObject(Inserter dst) {
+ var obj = dst.insertOBJECT();
+ for (int i = 0; i < numLeafs; ++i) {
+ assertTrue(insertLeaf(new ObjectInserter(obj, "leaf" + i), i).valid());
+ }
+ return obj;
+ }
+ static Cursor insertInnerArray(Inserter dst) {
+ var arr = dst.insertARRAY();
+ for (int i = 0; i < numLeafs; ++i) {
+ assertTrue(insertLeaf(new ArrayInserter(arr), i).valid());
+ }
+ return arr;
+ }
+ static Cursor insertOuterObject(Inserter dst) {
+ var obj = dst.insertOBJECT();
+ assertTrue(insertInnerObject(new ObjectInserter(obj, "foo")).valid());
+ assertTrue(insertInnerArray(new ObjectInserter(obj, "bar")).valid());
+ return obj;
+ }
+ static Cursor insertOuterArray(Inserter dst) {
+ var arr = dst.insertARRAY();
+ assertTrue(insertInnerObject(new ArrayInserter(arr)).valid());
+ assertTrue(insertInnerArray(new ArrayInserter(arr)).valid());
+ return arr;
+ }
+ static Cursor insertManySymbols(Inserter dst) {
+ var obj = dst.insertOBJECT();
+ for (int i = 0; i < 300; ++i) {
+ obj.setLong("val" + i, i);
+ }
+ assertEquals(300, obj.fields());
+ return obj;
+ }
+ static Cursor insertLargeArray(Inserter dst) {
+ var arr = dst.insertARRAY();
+ for (int i = 0; i < 300; ++i) {
+ arr.addLong(i);
+ }
+ assertEquals(300, arr.entries());
+ return arr;
+ }
+ static final int numShapes = numLeafs + 6;
+ static Cursor insertRoot(Slime dst, int shape) {
+ var root = new SlimeInserter(dst);
+ if (shape < numLeafs) {
+ return insertLeaf(root, shape);
+ }
+ return switch (shape) {
+ case (numLeafs) -> insertInnerObject(root);
+ case (numLeafs + 1) -> insertInnerArray(root);
+ case (numLeafs + 2) -> insertOuterObject(root);
+ case (numLeafs + 3) -> insertOuterArray(root);
+ case (numLeafs + 4) -> insertManySymbols(root);
+ case (numLeafs + 5) -> insertLargeArray(root);
+ default -> NixValue.invalid();
+ };
+ }
+ static Slime makeSlime(int shape) {
+ var slime = new Slime();
+ var root = insertRoot(slime, shape);
+ assertTrue(root.valid());
+ return slime;
+ }
+
+ class MyConsumer implements Consumer<Inspector> {
+ Inspector value = null;
+ @Override public void accept(Inspector value) {
+ assertNull(ctx, this.value);
+ this.value = value;
+ }
+ };
+
+ void checkConsumer(Inspector view) {
+ var consumer = new MyConsumer();
+ view.ifValid(consumer);
+ assertEquals(ctx, view.valid(), consumer.value != null);
+ if (view.valid()) {
+ assertSame(ctx, view, consumer.value);
+ }
+ }
+
+ class MyVisitor implements Visitor {
+ enum Called { NONE, INVALID, NIX, BOOL, LONG, DOUBLE, UTF8, DATA, ARRAY, OBJECT }
+ Called called = Called.NONE;
+ boolean boolValue;
+ long longValue;
+ double doubleValue;
+ byte[] bytes;
+ Inspector stuff;
+ @Override public void visitInvalid() {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.INVALID;
+ }
+ @Override public void visitNix() {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.NIX;
+ }
+ @Override public void visitBool(boolean bit) {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.BOOL;
+ boolValue = bit;
+ }
+ @Override public void visitLong(long l) {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.LONG;
+ longValue = l;
+ }
+ @Override public void visitDouble(double d) {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.DOUBLE;
+ doubleValue = d;
+ }
+ @Override public void visitString(String str) {
+ fail(ctx + ", strings are never utf-16 in binary view");
+ }
+ @Override public void visitString(byte[] utf8) {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.UTF8;
+ bytes = utf8;
+ }
+ @Override public void visitData(byte[] data) {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.DATA;
+ bytes = data;
+ }
+ @Override public void visitArray(Inspector arr) {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.ARRAY;
+ stuff = arr;
+ }
+ @Override public void visitObject(Inspector obj) {
+ assertEquals(ctx, Called.NONE, called);
+ called = Called.OBJECT;
+ stuff = obj;
+ }
+ };
+
+ void checkVisitor(Inspector view) {
+ var visitor = new MyVisitor();
+ view.accept(visitor);
+ if (!view.valid()) {
+ assertEquals(ctx, MyVisitor.Called.INVALID, visitor.called);
+ return;
+ }
+ switch (view.type()) {
+ case NIX:
+ assertEquals(ctx, MyVisitor.Called.NIX, visitor.called);
+ break;
+ case BOOL:
+ assertEquals(ctx, MyVisitor.Called.BOOL, visitor.called);
+ assertEquals(ctx, view.asBool(), visitor.boolValue);
+ break;
+ case LONG:
+ assertEquals(ctx, MyVisitor.Called.LONG, visitor.called);
+ assertEquals(ctx, view.asLong(), visitor.longValue);
+ break;
+ case DOUBLE:
+ assertEquals(ctx, MyVisitor.Called.DOUBLE, visitor.called);
+ assertEquals(ctx, view.asDouble(), visitor.doubleValue, 0.0);
+ break;
+ case STRING:
+ assertEquals(ctx, MyVisitor.Called.UTF8, visitor.called);
+ assertArrayEquals(ctx, view.asUtf8(), visitor.bytes);
+ break;
+ case DATA:
+ assertEquals(ctx, MyVisitor.Called.DATA, visitor.called);
+ assertArrayEquals(ctx, view.asData(), visitor.bytes);
+ break;
+ case ARRAY:
+ assertEquals(ctx, MyVisitor.Called.ARRAY, visitor.called);
+ assertSame(ctx, view, visitor.stuff);
+ break;
+ case OBJECT:
+ assertEquals(ctx, MyVisitor.Called.OBJECT, visitor.called);
+ assertSame(ctx, view, visitor.stuff);
+ break;
+ default:
+ fail(ctx + ", should not be reached");
+ break;
+ }
+ }
+
+ class MyArrayTraverser implements ArrayTraverser {
+ ArrayList<Inspector> list = new ArrayList<>();
+ @Override public void entry(int idx, Inspector value) {
+ list.add(value);
+ }
+ }
+
+ void checkTraverseArray(Inspector value, Inspector view) {
+ var a = new MyArrayTraverser();
+ var b = new MyArrayTraverser();
+ value.traverse(a);
+ view.traverse(b);
+ assertEquals(ctx, a.list.size(), b.list.size());
+ for (int i = 0; i < a.list.size(); ++i) {
+ checkParity(a.list.get(i), b.list.get(i));
+ }
+ }
+
+ class MyObjectSymbolTraverser implements ObjectSymbolTraverser {
+ HashMap<Integer,Inspector> map = new HashMap<>();
+ @Override public void field(int sym, Inspector value) {
+ map.put(sym, value);
+ }
+ }
+
+ void checkTraverseObjectSymbol(Inspector value, Inspector view) {
+ var a = new MyObjectSymbolTraverser();
+ var b = new MyObjectSymbolTraverser();
+ value.traverse(a);
+ view.traverse(b);
+ assertEquals(ctx, a.map.size(), b.map.size());
+ for (Integer key: a.map.keySet()) {
+ assertTrue(ctx, b.map.containsKey(key));
+ checkParity(a.map.get(key), b.map.get(key));
+ }
+ }
+
+ class MyObjectTraverser implements ObjectTraverser {
+ HashMap<String,Inspector> map = new HashMap<>();
+ @Override public void field(String name, Inspector value) {
+ map.put(name, value);
+ }
+ }
+
+ void checkTraverseObject(Inspector value, Inspector view) {
+ var a = new MyObjectTraverser();
+ var b = new MyObjectTraverser();
+ value.traverse(a);
+ view.traverse(b);
+ assertEquals(ctx, a.map.size(), b.map.size());
+ for (String key: a.map.keySet()) {
+ assertTrue(ctx, b.map.containsKey(key));
+ checkParity(a.map.get(key), b.map.get(key));
+ }
+ }
+ void checkParity(Inspector value, Inspector view) {
+ checkConsumer(view);
+ checkVisitor(view);
+ if (value == view) {
+ // avoid infinite invalid nix recursion
+ assertSame(ctx, value, view);
+ return;
+ }
+ assertEquals(ctx, value.valid(), view.valid());
+ assertEquals(ctx, value.type(), view.type());
+ assertEquals(ctx, value.children(), view.children());
+ assertEquals(ctx, value.entries(), view.entries());
+ assertEquals(ctx, value.fields(), view.fields());
+ assertEquals(ctx, value.asBool(), view.asBool());
+ assertEquals(ctx, value.asLong(), view.asLong());
+ assertEquals(ctx, value.asDouble(), view.asDouble(), 0.0);
+ assertEquals(ctx, value.asString(), view.asString());
+ assertArrayEquals(ctx, value.asUtf8(), view.asUtf8());
+ assertArrayEquals(ctx, value.asData(), view.asData());
+ checkTraverseArray(value, view);
+ checkTraverseObjectSymbol(value, view);
+ checkTraverseObject(value, view);
+ checkParity(value.entry(0), view.entry(0));
+ checkParity(value.entry(1), view.entry(1));
+ checkParity(value.entry(2), view.entry(2));
+ checkParity(value.entry(3), view.entry(3));
+ checkParity(value.entry(200), view.entry(200));
+ checkParity(value.entry(500), view.entry(500));
+ checkParity(value.entry(-1), view.entry(-1));
+ checkParity(value.field(0), view.field(0));
+ checkParity(value.field(1), view.field(1));
+ checkParity(value.field(2), view.field(2));
+ checkParity(value.field(3), view.field(3));
+ checkParity(value.field(SymbolTable.INVALID), view.field(SymbolTable.INVALID));
+ checkParity(value.field(-1), view.field(-1));
+ checkParity(value.field("foo"), view.field("foo"));
+ checkParity(value.field("bar"), view.field("bar"));
+ checkParity(value.field("val256"), view.field("val256"));
+ checkParity(value.field("bogus"), view.field("bogus"));
+ assertTrue(ctx, value.equalTo(view));
+ assertTrue(ctx, view.equalTo(value));
+ }
+
+ String ctx;
+ @Test public void testBinaryViewShapesParity() {
+ for (int i = 0; i < numShapes; ++i) {
+ var slime = makeSlime(i);
+ ctx = "case " + i + ": '" + slime.toString() + "'";
+ byte[] data = BinaryFormat.encode(slime);
+ try {
+ checkParity(slime.get(), BinaryView.inspect(data));
+ } catch (Exception e) {
+ fail(ctx + ", got exception: " + e);
+ }
+ }
+ }
+
+ @Test public void testTrivialView() {
+ byte[] data = {0, 0};
+ var input = new BufferedInput(data);
+ var view = BinaryView.inspectImpl(input);
+ assertTrue(view.valid());
+ assertEquals(Type.NIX, view.type());
+ assertFalse(input.failed());
+ }
+
+ @Test public void testUnderflow() {
+ byte[] data = {};
+ var input = new BufferedInput(data);
+ var view = BinaryView.inspectImpl(input);
+ assertFalse(view.valid());
+ assertTrue(input.failed());
+ assertEquals("underflow", input.getErrorMessage());
+ }
+
+ @Test public void testMultiByteUnderflow() {
+ byte[] data = { 0, encode_type_and_meta(Type.STRING.ID, 3), 65 };
+ var input = new BufferedInput(data);
+ var view = BinaryView.inspectImpl(input);
+ assertFalse(view.valid());
+ assertTrue(input.failed());
+ assertEquals("underflow", input.getErrorMessage());
+ }
+
+ @Test public void testCompressedIntOverflow() {
+ byte[] data = { -1, -1, -1, -1, 8 };
+ var input = new BufferedInput(data);
+ var view = BinaryView.inspectImpl(input);
+ assertFalse(view.valid());
+ assertTrue(input.failed());
+ assertEquals("compressed int overflow", input.getErrorMessage());
+ }
+
+ @Test public void testExtBitsOverflow() {
+ byte[] data = { 0, encode_type_and_meta(Type.OBJECT.ID, 2), -1, -1, -1, -1, 1 };
+ var input = new BufferedInput(data);
+ var view = BinaryView.inspectImpl(input);
+ assertFalse(view.valid());
+ assertTrue(input.failed());
+ assertEquals("symbol id too big", input.getErrorMessage());
+ }
+
+ @Test public void testDecodeIndexOverflowArray() {
+ byte[] data = { 0, encode_type_and_meta(Type.ARRAY.ID, 4) };
+ var input = new BufferedInput(data);
+ var view = BinaryView.inspectImpl(input);
+ assertFalse(view.valid());
+ assertTrue(input.failed());
+ assertEquals("decode index too big", input.getErrorMessage());
+ }
+
+ @Test public void testDecodeIndexOverflowObject() {
+ byte[] data = { 0, encode_type_and_meta(Type.OBJECT.ID, 4) };
+ var input = new BufferedInput(data);
+ var view = BinaryView.inspectImpl(input);
+ assertFalse(view.valid());
+ assertTrue(input.failed());
+ assertEquals("decode index too big", input.getErrorMessage());
+ }
+}
diff --git a/vespalib/src/vespa/vespalib/util/optimized.h b/vespalib/src/vespa/vespalib/util/optimized.h
index e78372b03ff..a85c1d18c00 100644
--- a/vespalib/src/vespa/vespalib/util/optimized.h
+++ b/vespalib/src/vespa/vespalib/util/optimized.h
@@ -31,7 +31,7 @@ public:
* @fn int Optimized::msbIdx(uint32_t v)
* @brief Quickly find most significant bit.
*
- * Finds the postion of the most significant '1'.
+ * Finds the position of the most significant '1'.
* @param v is the value to search
* @return index [0-31] of msb, 0 if none.
**/
@@ -40,7 +40,7 @@ public:
* @fn int Optimized::msbIdx(uint64_t v)
* @brief Quickly find most significant bit.
*
- * Finds the postion of the most significant '1'.
+ * Finds the position of the most significant '1'.
* @param v is the value to search
* @return index [0-63] of msb, 0 if none.
**/
@@ -49,7 +49,7 @@ public:
* @fn int Optimized::lsbIdx(uint32_t v)
* @brief Quickly find least significant bit.
*
- * Finds the postion of the least significant '1'.
+ * Finds the position of the least significant '1'.
* @param v is the value to search
* @return index [0-31] of lsb, 0 if none.
**/
@@ -58,7 +58,7 @@ public:
* @fn int Optimized::lsbIdx(uint64_t v)
* @brief Quickly find least significant bit.
*
- * Finds the postion of the least significant '1'.
+ * Finds the position of the least significant '1'.
* @param v is the value to search
* @return index [0-63] of lsb, 0 if none.
**/
diff --git a/zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Configurator.java b/zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Configurator.java
index af42e30422b..0b2595b6af8 100644
--- a/zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Configurator.java
+++ b/zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Configurator.java
@@ -2,20 +2,28 @@
package com.yahoo.vespa.zookeeper;
import com.yahoo.cloud.config.ZookeeperServerConfig;
+import com.yahoo.cloud.config.ZookeeperServerConfig.Server;
import com.yahoo.security.tls.ConfigFileBasedTlsContext;
import com.yahoo.security.tls.MixedMode;
import com.yahoo.security.tls.TlsContext;
import com.yahoo.security.tls.TransportSecurityUtils;
+
import java.io.FileWriter;
import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Arrays;
+import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.Optional;
+import java.util.Set;
import java.util.logging.Level;
import java.util.stream.Collectors;
+import static com.yahoo.stream.CustomCollectors.toLinkedMap;
import static com.yahoo.vespa.defaults.Defaults.getDefaults;
public class Configurator {
@@ -64,7 +72,6 @@ public class Configurator {
// override of Vespa TLS config for unit testing
void writeConfigToDisk(VespaTlsConfig vespaTlsConfig) {
configFilePath.toFile().getParentFile().mkdirs();
-
try {
writeZooKeeperConfigFile(zookeeperServerConfig, vespaTlsConfig);
writeMyIdFile(zookeeperServerConfig);
@@ -75,36 +82,75 @@ public class Configurator {
private void writeZooKeeperConfigFile(ZookeeperServerConfig config,
VespaTlsConfig vespaTlsConfig) throws IOException {
+ String dynamicConfigPath = config.dynamicReconfiguration() ? parseConfigFile(configFilePath).get("dynamicConfigFile") : null;
+ Map<String, String> dynamicConfig = dynamicConfigPath != null ? parseConfigFile(Paths.get(dynamicConfigPath)) : Map.of();
try (FileWriter writer = new FileWriter(configFilePath.toFile())) {
- writer.write(transformConfigToString(config, vespaTlsConfig));
+ writer.write(transformConfigToString(config, vespaTlsConfig, dynamicConfig));
}
}
- private String transformConfigToString(ZookeeperServerConfig config, VespaTlsConfig vespaTlsConfig) {
- StringBuilder sb = new StringBuilder();
- sb.append("tickTime=").append(config.tickTime()).append("\n");
- sb.append("initLimit=").append(config.initLimit()).append("\n");
- sb.append("syncLimit=").append(config.syncLimit()).append("\n");
- sb.append("maxClientCnxns=").append(config.maxClientConnections()).append("\n");
- sb.append("snapCount=").append(config.snapshotCount()).append("\n");
- sb.append("dataDir=").append(getDefaults().underVespaHome(config.dataDir())).append("\n");
- sb.append("autopurge.purgeInterval=").append(config.autopurge().purgeInterval()).append("\n");
- sb.append("autopurge.snapRetainCount=").append(config.autopurge().snapRetainCount()).append("\n");
+ private String transformConfigToString(ZookeeperServerConfig config, VespaTlsConfig vespaTlsConfig, Map<String, String> dynamicConfig) {
+ Map<String, String> configEntries = new LinkedHashMap<>();
+ configEntries.put("tickTime", Integer.toString(config.tickTime()));
+ configEntries.put("initLimit", Integer.toString(config.initLimit()));
+ configEntries.put("syncLimit", Integer.toString(config.syncLimit()));
+ configEntries.put("maxClientCnxns", Integer.toString(config.maxClientConnections()));
+ configEntries.put("snapCount", Integer.toString(config.snapshotCount()));
+ configEntries.put("dataDir", getDefaults().underVespaHome(config.dataDir()));
+ configEntries.put("autopurge.purgeInterval", Integer.toString(config.autopurge().purgeInterval()));
+ configEntries.put("autopurge.snapRetainCount", Integer.toString(config.autopurge().snapRetainCount()));
// See http://zookeeper.apache.org/doc/r3.6.3/zookeeperAdmin.html#sc_zkCommands
// Includes all available commands in 3.6, except 'wchc' and 'wchp'
- sb.append("4lw.commands.whitelist=conf,cons,crst,dirs,dump,envi,mntr,ruok,srst,srvr,stat,wchs").append("\n");
- sb.append("admin.enableServer=false").append("\n");
+ configEntries.put("4lw.commands.whitelist", "conf,cons,crst,dirs,dump,envi,mntr,ruok,srst,srvr,stat,wchs");
+ configEntries.put("admin.enableServer", "false");
// Use custom connection factory for TLS on client port - see class' Javadoc for rationale
- sb.append("serverCnxnFactory=org.apache.zookeeper.server.VespaNettyServerCnxnFactory").append("\n");
- sb.append("quorumListenOnAllIPs=true").append("\n");
- sb.append("standaloneEnabled=false").append("\n");
- sb.append("reconfigEnabled=").append(config.dynamicReconfiguration()).append("\n");
- sb.append("skipACL=yes").append("\n");
- ensureThisServerIsRepresented(config.myid(), config.server());
- config.server().forEach(server -> sb.append(serverSpec(server, server.joining())).append("\n"));
- sb.append(new TlsQuorumConfig().createConfig(vespaTlsConfig));
- sb.append(new TlsClientServerConfig().createConfig(vespaTlsConfig));
- return sb.toString();
+ configEntries.put("serverCnxnFactory", "org.apache.zookeeper.server.VespaNettyServerCnxnFactory");
+ configEntries.put("quorumListenOnAllIPs", "true");
+ configEntries.put("standaloneEnabled", "false");
+ configEntries.put("reconfigEnabled", Boolean.toString(config.dynamicReconfiguration()));
+ configEntries.put("skipACL", "yes");
+
+ addServerSpecs(configEntries, config, dynamicConfig);
+
+ new TlsQuorumConfig().createConfig(configEntries, vespaTlsConfig);
+ new TlsClientServerConfig().createConfig(configEntries, vespaTlsConfig);
+ return transformConfigToString(configEntries);
+ }
+
+ void addServerSpecs(Map<String, String> configEntries, ZookeeperServerConfig config, Map<String, String> dynamicConfig) {
+ int myIndex = ensureThisServerIsRepresented(config.myid(), config.server());
+
+ // If dynamic config refers to servers that are not in the current config, we must ignore it.
+ Set<String> currentServers = config.server().stream().map(Server::hostname).collect(Collectors.toSet());
+ if (dynamicConfig.values().stream().anyMatch(spec -> ! currentServers.contains(spec.split(":", 2)[0]))) {
+ log.log(Level.WARNING, "Existing dynamic config refers to unknown servers, ignoring it");
+ dynamicConfig = Map.of();
+ }
+
+ // If we have no existing, valid, dynamic config, we use all known servers as a starting point.
+ if (dynamicConfig.isEmpty()) {
+ configEntries.putAll(getServerConfig(config.server(), config.server(myIndex).joining() ? config.myid() : -1));
+ }
+ // Otherwise, we use the existing, dynamic config as a starting point, and add this as a joiner if not present.
+ else {
+ Map.Entry<String, String> thisAsAJoiner = getServerConfig(config.server().subList(myIndex, myIndex + 1), config.myid()).entrySet().iterator().next();
+ dynamicConfig.putIfAbsent(thisAsAJoiner.getKey(), thisAsAJoiner.getValue());
+ configEntries.putAll(dynamicConfig);
+ }
+
+ }
+ static Map<String, String> getServerConfig(List<ZookeeperServerConfig.Server> serversConfig, int joinerId) {
+ Map<String, String> configEntries = new LinkedHashMap<>();
+ for (Server server : serversConfig) {
+ configEntries.put("server." + server.id(), serverSpec(server, server.id() == joinerId));
+ }
+ return configEntries;
+ }
+
+ static String transformConfigToString(Map<String, String> config) {
+ return config.entrySet().stream()
+ .map(entry -> entry.getKey() + "=" + entry.getValue())
+ .collect(Collectors.joining("\n", "", "\n"));
}
private void writeMyIdFile(ZookeeperServerConfig config) throws IOException {
@@ -113,25 +159,17 @@ public class Configurator {
}
}
- private void ensureThisServerIsRepresented(int myid, List<ZookeeperServerConfig.Server> servers) {
- boolean found = false;
- for (ZookeeperServerConfig.Server server : servers) {
- if (myid == server.id()) {
- found = true;
- break;
- }
- }
- if (!found) {
- throw new RuntimeException("No id in zookeeper server list that corresponds to my id (" + myid + ")");
+ private static int ensureThisServerIsRepresented(int myid, List<ZookeeperServerConfig.Server> servers) {
+ for (int i = 0; i < servers.size(); i++) {
+ Server server = servers.get(i);
+ if (myid == server.id()) return i;
}
+ throw new RuntimeException("No id in zookeeper server list that corresponds to my id (" + myid + ")");
}
static String serverSpec(ZookeeperServerConfig.Server server, boolean joining) {
StringBuilder sb = new StringBuilder();
- sb.append("server.")
- .append(server.id())
- .append("=")
- .append(server.hostname())
+ sb.append(server.hostname())
.append(":")
.append(server.quorumPort())
.append(":")
@@ -150,6 +188,19 @@ public class Configurator {
return sb.toString();
}
+ static Map<String, String> parseConfigFile(Path configFilePath) {
+ try {
+ return Files.exists(configFilePath) ? Files.readAllLines(configFilePath).stream()
+ .filter(line -> ! line.startsWith("#"))
+ .map(line -> line.split("=", 2))
+ .collect(toLinkedMap(parts -> parts[0], parts -> parts[1]))
+ : Map.of();
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException("error reading zookeeper config", e);
+ }
+ }
+
static List<String> zookeeperServerHostnames(ZookeeperServerConfig zookeeperServerConfig) {
return zookeeperServerConfig.server().stream()
.map(ZookeeperServerConfig.Server::hostname)
@@ -165,15 +216,15 @@ public class Configurator {
private interface TlsConfig {
String configFieldPrefix();
- default void appendSharedTlsConfig(StringBuilder builder, VespaTlsConfig vespaTlsConfig) {
+ default void appendSharedTlsConfig(Map<String, String> configEntries, VespaTlsConfig vespaTlsConfig) {
vespaTlsConfig.context().ifPresent(ctx -> {
VespaSslContextProvider.set(ctx);
- builder.append(configFieldPrefix()).append(".context.supplier.class=").append(VespaSslContextProvider.class.getName()).append("\n");
+ configEntries.put(configFieldPrefix() + ".context.supplier.class", VespaSslContextProvider.class.getName());
String enabledCiphers = Arrays.stream(ctx.parameters().getCipherSuites()).sorted().collect(Collectors.joining(","));
- builder.append(configFieldPrefix()).append(".ciphersuites=").append(enabledCiphers).append("\n");
+ configEntries.put(configFieldPrefix() + ".ciphersuites", enabledCiphers);
String enabledProtocols = Arrays.stream(ctx.parameters().getProtocols()).sorted().collect(Collectors.joining(","));
- builder.append(configFieldPrefix()).append(".enabledProtocols=").append(enabledProtocols).append("\n");
- builder.append(configFieldPrefix()).append(".clientAuth=NEED\n");
+ configEntries.put(configFieldPrefix() + ".enabledProtocols", enabledProtocols);
+ configEntries.put(configFieldPrefix() + ".clientAuth", "NEED");
});
}
@@ -185,16 +236,13 @@ public class Configurator {
static class TlsClientServerConfig implements TlsConfig {
- public String createConfig(VespaTlsConfig vespaTlsConfig) {
- StringBuilder sb = new StringBuilder()
- .append("client.portUnification=").append(enablePortUnification(vespaTlsConfig)).append("\n");
+ public void createConfig(Map<String, String> configEntries, VespaTlsConfig vespaTlsConfig) {
+ configEntries.put("client.portUnification", String.valueOf(enablePortUnification(vespaTlsConfig)));
// ZooKeeper Dynamic Reconfiguration requires the "non-secure" client port to exist
// This is a hack to override the secure parameter through our connection factory wrapper
// https://issues.apache.org/jira/browse/ZOOKEEPER-3577
VespaNettyServerCnxnFactory_isSecure = vespaTlsConfig.tlsEnabled() && vespaTlsConfig.mixedMode() == MixedMode.DISABLED;
- appendSharedTlsConfig(sb, vespaTlsConfig);
-
- return sb.toString();
+ appendSharedTlsConfig(configEntries, vespaTlsConfig);
}
@Override
@@ -205,12 +253,10 @@ public class Configurator {
static class TlsQuorumConfig implements TlsConfig {
- public String createConfig(VespaTlsConfig vespaTlsConfig) {
- StringBuilder sb = new StringBuilder()
- .append("sslQuorum=").append(vespaTlsConfig.tlsEnabled()).append("\n")
- .append("portUnification=").append(enablePortUnification(vespaTlsConfig)).append("\n");
- appendSharedTlsConfig(sb, vespaTlsConfig);
- return sb.toString();
+ public void createConfig(Map<String, String> configEntries, VespaTlsConfig vespaTlsConfig) {
+ configEntries.put("sslQuorum", String.valueOf(vespaTlsConfig.tlsEnabled()));
+ configEntries.put("portUnification", String.valueOf(enablePortUnification(vespaTlsConfig)));
+ appendSharedTlsConfig(configEntries, vespaTlsConfig);
}
@Override
diff --git a/zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Reconfigurer.java b/zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Reconfigurer.java
index eb84b13d4d6..15431550d82 100644
--- a/zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Reconfigurer.java
+++ b/zookeeper-server/zookeeper-server-common/src/main/java/com/yahoo/vespa/zookeeper/Reconfigurer.java
@@ -6,15 +6,15 @@ import com.yahoo.component.AbstractComponent;
import com.yahoo.component.annotation.Inject;
import com.yahoo.protect.Process;
import com.yahoo.yolean.Exceptions;
+
import java.time.Duration;
import java.time.Instant;
-import java.util.List;
import java.util.Objects;
import java.util.function.Supplier;
import java.util.logging.Level;
import java.util.logging.Logger;
-import static com.yahoo.vespa.zookeeper.Configurator.serverSpec;
+import static java.util.stream.Collectors.joining;
/**
* Starts zookeeper server and supports reconfiguring zookeeper cluster. Keep this as a component
@@ -26,9 +26,11 @@ public class Reconfigurer extends AbstractComponent {
private static final Logger log = java.util.logging.Logger.getLogger(Reconfigurer.class.getName());
- private static final Duration TIMEOUT = Duration.ofMinutes(15);
+ static final Duration TIMEOUT = Duration.ofMinutes(15);
private final ExponentialBackoff backoff = new ExponentialBackoff(Duration.ofMillis(50), Duration.ofSeconds(10));
+ private final Duration timeout;
+ private final boolean haltOnFailure;
private final VespaZooKeeperAdmin vespaZooKeeperAdmin;
private final Sleeper sleeper;
@@ -38,12 +40,14 @@ public class Reconfigurer extends AbstractComponent {
@Inject
public Reconfigurer(VespaZooKeeperAdmin vespaZooKeeperAdmin) {
- this(vespaZooKeeperAdmin, new Sleeper());
+ this(vespaZooKeeperAdmin, new Sleeper(), true, TIMEOUT);
}
- Reconfigurer(VespaZooKeeperAdmin vespaZooKeeperAdmin, Sleeper sleeper) {
+ public Reconfigurer(VespaZooKeeperAdmin vespaZooKeeperAdmin, Sleeper sleeper, boolean haltOnFailure, Duration timeout) {
this.vespaZooKeeperAdmin = Objects.requireNonNull(vespaZooKeeperAdmin);
this.sleeper = Objects.requireNonNull(sleeper);
+ this.haltOnFailure = haltOnFailure;
+ this.timeout = timeout;
}
@Override
@@ -86,14 +90,15 @@ public class Reconfigurer extends AbstractComponent {
// TODO jonmv: wrap Curator in Provider, for Curator shutdown
private void reconfigure(ZookeeperServerConfig newConfig) {
Instant reconfigTriggered = Instant.now();
- String newServers = String.join(",", servers(newConfig));
+ String newServers = servers(newConfig);
log.log(Level.INFO, "Will reconfigure ZooKeeper cluster." +
"\nServers in active config:" + servers(activeConfig) +
- "\nServers in new config:" + servers(newConfig));
+ "\nServers in new config:" + newServers);
String connectionSpec = vespaZooKeeperAdmin.localConnectionSpec(activeConfig);
Instant now = Instant.now();
- Duration reconfigTimeout = reconfigTimeout();
- Instant end = now.plus(reconfigTimeout);
+ // For reconfig to succeed, the current and resulting ensembles must have a majority. When an ensemble grows and
+ // the joining servers outnumber the existing ones, we have to wait for enough of them to start to have a majority.
+ Instant end = now.plus(timeout);
// Loop reconfiguring since we might need to wait until another reconfiguration is finished before we can succeed
for (int attempt = 1; ; attempt++) {
try {
@@ -116,29 +121,20 @@ public class Reconfigurer extends AbstractComponent {
}
else {
log.log(Level.SEVERE, "Reconfiguration attempt " + attempt + " failed, and was failing for " +
- reconfigTimeout + "; giving up now: " + Exceptions.toMessageString(e));
- shutdownAndDie(reconfigTimeout);
+ timeout + "; giving up now: " + Exceptions.toMessageString(e));
+ shutdown();
+ if (haltOnFailure)
+ Process.logAndDie("Reconfiguration did not complete within timeout " + timeout + ". Forcing container shutdown.");
+ else
+ throw e;
}
}
}
}
- private void shutdownAndDie(Duration reconfigTimeout) {
- shutdown();
- Process.logAndDie("Reconfiguration did not complete within timeout " + reconfigTimeout + ". Forcing container shutdown.");
- }
-
- private static Duration reconfigTimeout() {
- // For reconfig to succeed, the current and resulting ensembles must have a majority. When an ensemble grows and
- // the joining servers outnumber the existing ones, we have to wait for enough of them to start to have a majority.
- return TIMEOUT;
- }
-
- private static List<String> servers(ZookeeperServerConfig config) {
- return config.server().stream()
- .filter(server -> ! server.retired())
- .map(server -> serverSpec(server, false))
- .toList();
+ private static String servers(ZookeeperServerConfig config) {
+ return Configurator.getServerConfig(config.server().stream().filter(server -> ! server.retired()).toList(), -1)
+ .entrySet().stream().map(entry -> entry.getKey() + "=" + entry.getValue()).collect(joining(","));
}
}
diff --git a/zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ConfiguratorTest.java b/zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ConfiguratorTest.java
index 5d0031d5b55..08acbf2b838 100644
--- a/zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ConfiguratorTest.java
+++ b/zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ConfiguratorTest.java
@@ -24,7 +24,10 @@ import java.math.BigInteger;
import java.nio.file.Files;
import java.security.KeyPair;
import java.security.cert.X509Certificate;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import static com.yahoo.security.KeyAlgorithm.EC;
@@ -62,12 +65,12 @@ public class ConfiguratorTest {
}
@Test
- public void config_is_written_correctly_with_multiple_servers() {
+ public void config_is_written_correctly_with_multiple_servers() throws IOException {
three_config_servers(false);
}
@Test
- public void config_is_written_correctly_with_multiple_servers_on_hosted_vespa() {
+ public void config_is_written_correctly_with_multiple_servers_on_hosted_vespa() throws IOException {
three_config_servers(true);
}
@@ -117,13 +120,49 @@ public class ConfiguratorTest {
assertEquals("" + max_buffer, System.getProperty(ZOOKEEPER_JUTE_MAX_BUFFER));
}
- private void three_config_servers(boolean hosted) {
+ @Test
+ public void test_parsing_config() throws IOException {
ZookeeperServerConfig.Builder builder = new ZookeeperServerConfig.Builder();
builder.zooKeeperConfigFile(cfgFile.getAbsolutePath());
builder.server(newServer(0, "foo", 123, 321, false));
builder.server(newServer(1, "bar", 234, 432, false));
builder.server(newServer(2, "baz", 345, 543, true));
builder.myidFile(idFile.getAbsolutePath());
+ builder.myid(2);
+ builder.tickTime(1234);
+ builder.dynamicReconfiguration(true);
+ Configurator configurator = new Configurator(builder.build());
+ configurator.writeConfigToDisk(VespaTlsConfig.tlsDisabled());
+ validateIdFile(idFile, "2\n");
+
+ assertEquals(Files.readString(cfgFile.toPath()),
+ Configurator.transformConfigToString(Configurator.parseConfigFile(cfgFile.toPath())));
+
+ Map<String, String> originalConfig = Configurator.parseConfigFile(cfgFile.toPath());
+ Map<String, String> staticConfig = new LinkedHashMap<>(originalConfig);
+ // Dynamic config says this is not a joiner.
+ Map<String, String> dynamicConfig = Configurator.getServerConfig(builder.build().server(), -1);
+ staticConfig.keySet().removeAll(dynamicConfig.keySet());
+ assertEquals(originalConfig.size(), dynamicConfig.size() + staticConfig.size());
+ File dynFile = folder.newFile();
+ staticConfig.put("dynamicConfigFile", dynFile.getAbsolutePath());
+ Files.write(cfgFile.toPath(), Configurator.transformConfigToString(staticConfig).getBytes());
+ Files.write(dynFile.toPath(), Configurator.transformConfigToString(dynamicConfig).getBytes());
+
+ configurator.writeConfigToDisk(VespaTlsConfig.tlsDisabled());
+ // Next generation of config should not mark this as a joiner either.
+ originalConfig.putAll(Configurator.getServerConfig(builder.build().server().subList(2, 3), -1));
+ assertEquals(Configurator.transformConfigToString(originalConfig),
+ Files.readString(cfgFile.toPath()));
+ }
+
+ private void three_config_servers(boolean hosted) throws IOException {
+ ZookeeperServerConfig.Builder builder = new ZookeeperServerConfig.Builder();
+ builder.zooKeeperConfigFile(cfgFile.getAbsolutePath());
+ builder.server(newServer(0, "foo", 123, 321, false));
+ builder.server(newServer(1, "bar", 234, 432, true));
+ builder.server(newServer(2, "baz", 345, 543, false));
+ builder.myidFile(idFile.getAbsolutePath());
builder.myid(1);
builder.tickTime(1234);
builder.dynamicReconfiguration(hosted);
@@ -205,8 +244,8 @@ public class ConfiguratorTest {
String expected =
commonConfig(hosted) +
"server.0=foo:321:123;2181\n" +
- "server.1=bar:432:234;2181\n" +
- "server.2=baz:543:345:observer;2181\n" +
+ "server.1=bar:432:234:observer;2181\n" +
+ "server.2=baz:543:345;2181\n" +
"sslQuorum=false\n" +
"portUnification=false\n" +
"client.portUnification=false\n";
diff --git a/zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ReconfigurerTest.java b/zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ReconfigurerTest.java
index 697fba3b4c4..ea8dcac945c 100644
--- a/zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ReconfigurerTest.java
+++ b/zookeeper-server/zookeeper-server-common/src/test/java/com/yahoo/vespa/zookeeper/ReconfigurerTest.java
@@ -144,11 +144,13 @@ public class ReconfigurerTest {
TestableReconfigurer(TestableVespaZooKeeperAdmin zooKeeperAdmin) {
super(zooKeeperAdmin, new Sleeper() {
- @Override
- public void sleep(Duration duration) {
- // Do nothing
- }
- });
+ @Override
+ public void sleep(Duration duration) {
+ // Do nothing
+ }
+ },
+ false,
+ Reconfigurer.TIMEOUT);
this.zooKeeperAdmin = zooKeeperAdmin;
HostName.setHostNameForTestingOnly("node1");
}