forked from apache/cassandra-gocql-driver
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathsession.go
More file actions
2493 lines (2148 loc) · 80.4 KB
/
session.go
File metadata and controls
2493 lines (2148 loc) · 80.4 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/*
* Content before git sha 34fdeebefcbf183ed7f916f931aa0586fdaa1b40
* Copyright (c) 2012, The Gocql authors,
* provided under the BSD-3-Clause License.
* See the NOTICE file distributed with this work for additional information.
*/
package gocql
import (
"bytes"
"context"
"encoding/binary"
"errors"
"fmt"
"io"
"net"
"strings"
"sync"
"sync/atomic"
"time"
"unicode"
"github.com/apache/cassandra-gocql-driver/v2/internal/lru"
)
// Session is the interface used by users to interact with the database.
//
// It's safe for concurrent use by multiple goroutines and a typical usage
// scenario is to have one global session object to interact with the
// whole Cassandra cluster.
//
// This type extends the Node interface by adding a convenient query builder
// and automatically sets a default consistency level on all operations
// that do not have a consistency level set.
type Session struct {
cons Consistency
pageSize int
prefetch float64
routingMetadataCache routingKeyInfoLRU
schemaDescriber *schemaDescriber
trace Tracer
queryObserver QueryObserver
batchObserver BatchObserver
connectObserver ConnectObserver
frameObserver FrameHeaderObserver
streamObserver StreamObserver
hostSource *ringDescriber
ringRefresher *refreshDebouncer
stmtsLRU *preparedLRU
types *RegisteredTypes
connCfg *ConnConfig
executor *queryExecutor
pool *policyConnPool
policy HostSelectionPolicy
ring ring
metadata clusterMetadata
control *controlConn
// event handlers
nodeEvents *eventDebouncer
// host state and topology change listeners
hostListeners internalHostListeners
// schema change listeners
schemaListeners internalSchemaListeners
// session ready listeners
sessionReadyListeners internalSessionReadyListener
// ring metadata
useSystemSchema bool
hasAggregatesAndFunctions bool
cfg ClusterConfig
ctx context.Context
cancel context.CancelFunc
// sessionStateMu protects isClosed and isInitialized.
sessionStateMu sync.RWMutex
// isClosed is true once Session.Close is finished.
isClosed bool
// isClosing bool is true once Session.Close is started.
isClosing bool
// isInitialized is true once Session.init succeeds.
// you can use initialized() to read the value.
isInitialized bool
logger StructuredLogger
}
func addrsToHosts(addrs []string, defaultPort int, logger StructuredLogger) ([]*HostInfo, error) {
var hosts []*HostInfo
for _, hostaddr := range addrs {
resolvedHosts, err := hostInfo(hostaddr, defaultPort)
if err != nil {
// Try other hosts if unable to resolve DNS name
if _, ok := err.(*net.DNSError); ok {
logger.Error("DNS error.", NewLogFieldError("err", err))
continue
}
return nil, err
}
hosts = append(hosts, resolvedHosts...)
}
if len(hosts) == 0 {
return nil, errors.New("failed to resolve any of the provided hostnames")
}
return hosts, nil
}
// NewSession wraps an existing Node.
func NewSession(cfg ClusterConfig) (*Session, error) {
// Check that hosts in the ClusterConfig is not empty
if len(cfg.Hosts) < 1 {
return nil, ErrNoHosts
}
// Check that either Authenticator is set or AuthProvider, not both
if cfg.Authenticator != nil && cfg.AuthProvider != nil {
return nil, errors.New("Can't use both Authenticator and AuthProvider in cluster config.")
}
if cfg.SerialConsistency > 0 && !cfg.SerialConsistency.isSerial() {
return nil, fmt.Errorf("the default SerialConsistency level is not allowed to be anything else but SERIAL or LOCAL_SERIAL. Recived value: %v", cfg.SerialConsistency)
}
// TODO: we should take a context in here at some point
ctx, cancel := context.WithCancel(context.TODO())
s := &Session{
cons: cfg.Consistency,
prefetch: cfg.NextPagePrefetch,
cfg: cfg,
pageSize: cfg.PageSize,
stmtsLRU: &preparedLRU{lru: lru.New(cfg.MaxPreparedStmts)},
connectObserver: cfg.ConnectObserver,
ctx: ctx,
cancel: cancel,
logger: cfg.newLogger(),
trace: cfg.Tracer,
}
if cfg.RegisteredTypes == nil {
s.types = GlobalTypes.Copy()
} else {
s.types = cfg.RegisteredTypes.Copy()
}
s.schemaDescriber = newSchemaDescriber(s, newRefreshDebouncer(schemaRefreshDebounceTime, func() error {
return refreshSchemas(s)
}))
s.nodeEvents = newEventDebouncer("NodeEvents", s.handleNodeEvent, s.logger)
s.routingMetadataCache.lru = lru.New(cfg.MaxRoutingKeyInfo)
s.hostSource = &ringDescriber{session: s}
s.ringRefresher = newRefreshDebouncer(ringRefreshDebounceTime, func() error { return refreshRing(s.hostSource) })
s.queryObserver = cfg.QueryObserver
s.batchObserver = cfg.BatchObserver
s.connectObserver = cfg.ConnectObserver
s.frameObserver = cfg.FrameHeaderObserver
s.streamObserver = cfg.StreamObserver
// Propogate node status, topology and schema change listeners
s.hostListeners = newInternalHostStateListeners(
s,
cfg.Metadata.HostListener.HostStateChangeListener,
cfg.Metadata.HostListener.TopologyChangeListener,
)
// Propogate schema change listeners
s.schemaListeners = newInternalSchemaChangeListeners(
cfg.Metadata.SchemaListener.KeyspaceChangeListener,
cfg.Metadata.SchemaListener.TableChangeListener,
cfg.Metadata.SchemaListener.UserTypeChangeListener,
cfg.Metadata.SchemaListener.FunctionChangeListener,
cfg.Metadata.SchemaListener.AggregateChangeListener,
)
if cfg.Metadata.CacheMode == Disabled && s.schemaListeners.hasSchemaChangeListeners() {
return nil, errors.New("Schema change listeners are not supported in Disabled metadata cache mode")
}
if cfg.Metadata.CacheMode == KeyspaceOnly && s.schemaListeners.hasNonKeyspaceSchemaChangeListeners() {
return nil, errors.New("Schema change listeners are not supported in KeyspaceOnly metadata cache mode")
}
// Propogate session ready listener
s.sessionReadyListeners = newInternalSessionReadyListener(cfg.Metadata.SessionReadyListener)
//Check the TLS Config before trying to connect to anything external
connCfg, err := connConfig(&s.cfg)
if err != nil {
//TODO: Return a typed error
return nil, fmt.Errorf("gocql: unable to create session: %v", err)
}
s.connCfg = connCfg
if cfg.PoolConfig.HostSelectionPolicy == nil {
cfg.PoolConfig.HostSelectionPolicy = RoundRobinHostPolicy()
}
s.pool = cfg.PoolConfig.buildPool(s)
s.policy = cfg.PoolConfig.HostSelectionPolicy
// set the executor here in case the policy needs to execute queries in Init
s.executor = &queryExecutor{
pool: s.pool,
policy: cfg.PoolConfig.HostSelectionPolicy,
}
s.policy.Init(s)
if err := s.init(); err != nil {
s.Close()
if err == ErrNoConnectionsStarted {
//This error used to be generated inside NewSession & returned directly
//Forward it on up to be backwards compatible
return nil, ErrNoConnectionsStarted
} else {
// TODO(zariel): dont wrap this error in fmt.Errorf, return a typed error
return nil, fmt.Errorf("gocql: unable to create session: %v", err)
}
}
return s, nil
}
func (s *Session) init() error {
hosts, err := addrsToHosts(s.cfg.Hosts, s.cfg.Port, s.logger)
if err != nil {
return err
}
s.ring.endpoints = hosts
if !s.cfg.disableControlConn {
s.control = createControlConn(s)
if s.cfg.ProtoVersion == 0 {
proto, err := s.control.discoverProtocol(hosts)
if err != nil {
return fmt.Errorf("unable to discover protocol version: %v", err)
} else if proto == 0 {
return errors.New("unable to discovery protocol version")
}
// TODO(zariel): we really only need this in 1 place
s.cfg.ProtoVersion = proto
s.connCfg.ProtoVersion = proto
s.logger.Info("Discovered protocol version.", NewLogFieldInt("protocol_version", proto))
}
if err := s.control.connect(hosts, true); err != nil {
return err
}
if !s.cfg.DisableInitialHostLookup {
var partitioner string
newHosts, partitioner, err := s.hostSource.GetHosts()
if err != nil {
return err
}
s.policy.SetPartitioner(partitioner)
filteredHosts := make([]*HostInfo, 0, len(newHosts))
for _, host := range newHosts {
if !s.cfg.filterHost(host) {
filteredHosts = append(filteredHosts, host)
}
}
hosts = filteredHosts
s.logger.Info("Refreshed ring.", NewLogFieldString("ring", ringString(hosts)))
} else {
s.logger.Info("Not performing a ring refresh because DisableInitialHostLookup is true.")
}
}
for _, host := range hosts {
// In case when host lookup is disabled and when we are in unit tests,
// host are not discovered, and we are missing host ID information used
// by internal logic.
// Associate random UUIDs here with all hosts missing this information.
if len(host.HostID()) == 0 {
host.setHostID(MustRandomUUID().String())
}
}
hostMap := make(map[string]*HostInfo, len(hosts))
for _, host := range hosts {
hostMap[host.HostID()] = host
}
hosts = hosts[:0]
// each host will increment left and decrement it after connecting and once
// there's none left, we'll close hostCh
var left int64
// we will receive up to len(hostMap) of messages so create a buffer so we
// don't end up stuck in a goroutine if we stopped listening
connectedCh := make(chan struct{}, len(hostMap))
// we add one here because we don't want to end up closing hostCh until we're
// done looping and the decerement code might be reached before we've looped
// again
atomic.AddInt64(&left, 1)
for _, host := range hostMap {
host, exists := s.ring.addOrUpdate(host)
if s.cfg.filterHost(host) {
continue
}
if !exists {
s.logger.Info("Adding host (session initialization).",
NewLogFieldIP("host_addr", host.ConnectAddress()), NewLogFieldString("host_id", host.HostID()))
}
atomic.AddInt64(&left, 1)
go func() {
s.pool.addHost(host)
connectedCh <- struct{}{}
// if there are no hosts left, then close the hostCh to unblock the loop
// below if its still waiting
if atomic.AddInt64(&left, -1) == 0 {
close(connectedCh)
}
}()
hosts = append(hosts, host)
}
// once we're done looping we subtract the one we initially added and check
// to see if we should close
if atomic.AddInt64(&left, -1) == 0 {
close(connectedCh)
}
// before waiting for them to connect, add them all to the policy so we can
// utilize efficiencies by calling AddHosts if the policy supports it
type bulkAddHosts interface {
AddHosts([]*HostInfo)
}
if v, ok := s.policy.(bulkAddHosts); ok {
v.AddHosts(hosts)
} else {
for _, host := range hosts {
s.policy.AddHost(host)
}
}
readyPolicy, _ := s.policy.(ReadyPolicy)
// now loop over connectedCh until it's closed (meaning we've connected to all)
// or until the policy says we're ready
for range connectedCh {
if readyPolicy != nil && readyPolicy.Ready() {
break
}
}
// TODO(zariel): we probably dont need this any more as we verify that we
// can connect to one of the endpoints supplied by using the control conn.
// See if there are any connections in the pool
if s.cfg.ReconnectInterval > 0 {
go s.reconnectDownedHosts(s.cfg.ReconnectInterval)
}
// If we disable the initial host lookup, we need to still check if the
// cluster is using the newer system schema or not... however, if control
// connection is disable, we really have no choice, so we just make our
// best guess...
if !s.cfg.disableControlConn && s.cfg.DisableInitialHostLookup {
newer, _ := checkSystemSchema(s.control)
s.useSystemSchema = newer
} else {
version := s.ring.rrHost().Version()
s.useSystemSchema = version.AtLeast(3, 0, 0)
s.hasAggregatesAndFunctions = version.AtLeast(2, 2, 0)
}
if s.pool.Size() == 0 {
return ErrNoConnectionsStarted
}
// Invoke KeyspaceChanged to let the policy cache the session keyspace
// parameters. This is used by tokenAwareHostPolicy to discover replicas.
if !s.cfg.disableControlConn && s.schemaDescriber != nil {
err := s.schemaDescriber.refreshSchemaMetadata()
if err != nil {
s.logger.Warning("Failed to initialize schema metadata. "+
"Token-aware routing will fall back to the configured fallback policy. "+
"Attempts to retrieve keyspace metadata will fail with ErrKeyspaceDoesNotExist until schema refresh succeeds.",
NewLogFieldError("err", err))
}
}
s.sessionStateMu.Lock()
s.isInitialized = true
s.sessionStateMu.Unlock()
s.sessionReadyListeners.OnSessionReady(s)
s.logger.Info("Session initialized successfully.")
return nil
}
// AwaitSchemaAgreement will wait until schema versions across all nodes in the
// cluster are the same (as seen from the point of view of the control connection).
// The maximum amount of time this takes is governed
// by the MaxWaitSchemaAgreement setting in the configuration (default: 60s).
// AwaitSchemaAgreement returns an error in case schema versions are not the same
// after the timeout specified in MaxWaitSchemaAgreement elapses.
func (s *Session) AwaitSchemaAgreement(ctx context.Context) error {
if s.cfg.disableControlConn {
return errNoControl
}
return s.control.withConn(func(conn *Conn) *Iter {
return newErrIter(conn.awaitSchemaAgreement(ctx), &queryMetrics{}, "", nil, nil)
}).err
}
func (s *Session) reconnectDownedHosts(intv time.Duration) {
reconnectTicker := time.NewTicker(intv)
defer reconnectTicker.Stop()
for {
select {
case <-reconnectTicker.C:
s.logger.Debug("Connecting to downed hosts if there is any.")
hosts := s.ring.allHosts()
// Print session.ring for debug.
s.logger.Debug("Logging current ring state.", NewLogFieldString("ring", ringString(hosts)))
for _, h := range hosts {
if h.IsUp() {
continue
}
s.logger.Debug("Reconnecting to downed host.",
NewLogFieldIP("host_addr", h.ConnectAddress()),
NewLogFieldInt("host_port", h.Port()),
NewLogFieldString("host_id", h.HostID()))
// we let the pool call handleNodeConnected to change the host state
s.pool.addHost(h)
}
case <-s.ctx.Done():
return
}
}
}
// Query generates a new query object for interacting with the database.
// Further details of the query may be tweaked using the resulting query
// value before the query is executed. Query is automatically prepared
// if it has not previously been executed.
//
// Supported Go to CQL type conversions for query parameters are as follows:
//
// Go type (value) | CQL type | Note
// string, []byte | varchar, ascii, blob, text |
// bool | boolean |
// integer types | tinyint, smallint, int |
// string | tinyint, smallint, int | formatted as base 10 number
// integer types | bigint, counter |
// big.Int | bigint, counter | according to cassandra bigint specification the big.Int value limited to int64 size(an eight-byte two's complement integer.)
// string | bigint, counter | formatted as base 10 number
// float32 | float |
// float64 | double |
// inf.Dec | decimal |
// int64 | time | nanoseconds since start of day
// time.Duration | time | duration since start of day
// int64 | timestamp | milliseconds since Unix epoch
// time.Time | timestamp |
// slice, array | list, set |
// map[X]struct{} | list, set |
// map[X]Y | map |
// gocql.UUID | uuid, timeuuid |
// [16]byte | uuid, timeuuid | raw UUID bytes
// []byte | uuid, timeuuid | raw UUID bytes, length must be 16 bytes
// string | uuid, timeuuid | hex representation, see ParseUUID
// integer types | varint |
// big.Int | varint |
// string | varint | value of number in decimal notation
// net.IP | inet |
// string | inet | IPv4 or IPv6 address string
// slice, array | tuple |
// struct | tuple | fields are marshaled in order of declaration
// gocql.UDTMarshaler | user-defined type | MarshalUDT is called
// map[string]interface{} | user-defined type |
// struct | user-defined type | struct fields' cql tags are used for column names
// int64 | date | milliseconds since Unix epoch to start of day (in UTC)
// time.Time | date | start of day (in UTC)
// string | date | parsed using "2006-01-02" format
// int64 | duration | duration in nanoseconds
// time.Duration | duration |
// gocql.Duration | duration |
// string | duration | parsed with time.ParseDuration
func (s *Session) Query(stmt string, values ...interface{}) *Query {
qry := &Query{}
qry.session = s
qry.stmt = stmt
qry.values = values
qry.hostID = ""
qry.defaultsFromSession()
return qry
}
// QueryInfo represents metadata information about a prepared query.
// It contains the query ID, argument information, result information, and primary key columns.
type QueryInfo struct {
Id []byte
Args []ColumnInfo
Rval []ColumnInfo
PKeyColumns []int
}
// Bind generates a new query object based on the query statement passed in.
// The query is automatically prepared if it has not previously been executed.
// The binding callback allows the application to define which query argument
// values will be marshalled as part of the query execution.
// During execution, the meta data of the prepared query will be routed to the
// binding callback, which is responsible for producing the query argument values.
//
// For supported Go to CQL type conversions for query parameters, see Session.Query documentation.
func (s *Session) Bind(stmt string, b func(q *QueryInfo) ([]interface{}, error)) *Query {
qry := &Query{}
qry.session = s
qry.stmt = stmt
qry.binding = b
qry.defaultsFromSession()
return qry
}
// Close closes all connections. The session is unusable after this
// operation.
func (s *Session) Close() {
s.sessionStateMu.Lock()
if s.isClosing {
s.sessionStateMu.Unlock()
return
}
s.isClosing = true
s.sessionStateMu.Unlock()
if s.pool != nil {
s.pool.Close()
}
if s.schemaDescriber != nil {
s.schemaDescriber.schemaRefresher.stop()
}
if s.control != nil {
s.control.close()
}
if s.nodeEvents != nil {
s.nodeEvents.stop()
}
if s.ringRefresher != nil {
s.ringRefresher.stop()
}
if s.cancel != nil {
s.cancel()
}
s.sessionStateMu.Lock()
s.isClosed = true
s.sessionStateMu.Unlock()
}
func (s *Session) Closed() bool {
s.sessionStateMu.RLock()
closed := s.isClosed
s.sessionStateMu.RUnlock()
return closed
}
func (s *Session) initialized() bool {
s.sessionStateMu.RLock()
initialized := s.isInitialized
s.sessionStateMu.RUnlock()
return initialized
}
func (s *Session) executeQuery(qry *internalQuery) (it *Iter) {
// fail fast
if s.Closed() {
return newErrIter(ErrSessionClosed, qry.metrics, qry.Keyspace(), qry.getRoutingInfo(), qry.getKeyspaceFunc())
}
iter, err := s.executor.executeQuery(qry)
if err != nil {
return newErrIter(err, qry.metrics, qry.Keyspace(), qry.getRoutingInfo(), qry.getKeyspaceFunc())
}
if iter == nil {
panic("nil iter")
}
return iter
}
func (s *Session) removeHost(h *HostInfo) {
s.logger.Warning("Removing host.", NewLogFieldIP("host_addr", h.ConnectAddress()), NewLogFieldString("host_id", h.HostID()))
s.policy.RemoveHost(h)
hostID := h.HostID()
s.pool.removeHost(hostID)
s.ring.removeHost(hostID)
}
// KeyspaceMetadata returns the schema metadata for the keyspace specified. Returns an error if the keyspace does not exist.
// If MetadataConfig.CacheMode is Disabled this method will query the system tables,
// otherwise it will retrieve the metadata from the driver's cache.
//
// Check AllKeyspaceMetadata if you're interested in retrieving the metadata for all keyspaces instead.
func (s *Session) KeyspaceMetadata(keyspace string) (*KeyspaceMetadata, error) {
// fail fast
if s.Closed() {
return nil, ErrSessionClosed
} else if keyspace == "" {
return nil, ErrNoKeyspace
}
return s.schemaDescriber.getSchema(keyspace)
}
// AllKeyspaceMetadata returns the schema metadata for all keyspaces.
// If MetadataConfig.CacheMode is Disabled this method will query the system tables,
// otherwise it will retrieve the metadata from the driver's cache.
//
// Check KeyspaceMetadata if you're interested in retrieving the metadata for a single keyspace by name instead.
func (s *Session) AllKeyspaceMetadata() (map[string]*KeyspaceMetadata, error) {
// fail fast
if s.Closed() {
return nil, ErrSessionClosed
}
return s.schemaDescriber.getAllSchema()
}
func (s *Session) getConn() *Conn {
hosts := s.ring.allHosts()
for _, host := range hosts {
if !host.IsUp() {
continue
}
pool, ok := s.pool.getPool(host)
if !ok {
continue
} else if conn := pool.Pick(); conn != nil {
return conn
}
}
return nil
}
// Returns statement metadata for the purposes of generating a routing key.
// If keyspace == "" it uses the keyspace which is specified in Cluster.Keyspace
func (s *Session) routingStatementMetadata(ctx context.Context, stmt string, keyspace string) (*StatementMetadata, error) {
if keyspace == "" {
keyspace = s.cfg.Keyspace
}
key := keyspace + stmt
s.routingMetadataCache.mu.Lock()
// Using here keyspace + stmt as a cache key because
// the query keyspace could be overridden via SetKeyspace
entry, cached := s.routingMetadataCache.lru.Get(key)
if cached {
// done accessing the cache
s.routingMetadataCache.mu.Unlock()
// the entry is an inflight struct similar to that used by
// Conn to prepare statements
inflight := entry.(*inflightCachedEntry)
// wait for any inflight work
inflight.wg.Wait()
if inflight.err != nil {
return nil, inflight.err
}
key, _ := inflight.value.(*StatementMetadata)
return key, nil
}
// create a new inflight entry while the data is created
inflight := new(inflightCachedEntry)
inflight.wg.Add(1)
defer inflight.wg.Done()
s.routingMetadataCache.lru.Add(key, inflight)
s.routingMetadataCache.mu.Unlock()
var meta StatementMetadata
meta, inflight.err = s.StatementMetadata(ctx, stmt, keyspace)
if inflight.err != nil {
// don't cache this error
s.routingMetadataCache.Remove(key)
return nil, inflight.err
}
inflight.value = &meta
return &meta, nil
}
// StatementMetadata represents various metadata about a statement.
type StatementMetadata struct {
// Keyspace is the keyspace of the table for the statement.
Keyspace string
// Table is the table of the statement.
Table string
// BindColumns are columns bound to the statement.
BindColumns []ColumnInfo
// PKBindColumnIndexes are the indexes of the BindColumns that correspond to
// partition key columns. If this is empty then one or more columns in the
// partition key were not bound to the statement.
PKBindColumnIndexes []int
// ResultColumns are the columns that are returned by the statement.
ResultColumns []ColumnInfo
}
// StatementMetadata returns metadata for a statement. If keyspace is empty,
// the session's keyspace is used.
func (s *Session) StatementMetadata(ctx context.Context, stmt, keyspace string) (StatementMetadata, error) {
if keyspace == "" {
keyspace = s.cfg.Keyspace
}
conn := s.getConn()
if conn == nil {
return StatementMetadata{}, ErrNoConnections
}
// get the query info for the statement
info, err := conn.prepareStatement(ctx, stmt, nil, keyspace)
if err != nil {
// TODO: it would be nice to mark hosts here but as we are not using the policies
// to fetch hosts we cant and we can't use the policies because they might
// require token awareness which requires this method
return StatementMetadata{}, err
}
if info.request.keyspace != "" {
keyspace = info.request.keyspace
}
meta := StatementMetadata{
Keyspace: keyspace,
Table: info.request.table,
BindColumns: info.request.columns,
PKBindColumnIndexes: info.request.pkeyColumns,
ResultColumns: info.response.columns,
}
// if it is protocol < v4 then we need to calculate the routing key info
if !info.request.supportsPKeyColumns && len(info.request.columns) > 0 {
keyspaceMetadata, err := s.KeyspaceMetadata(meta.Keyspace)
if err != nil {
// don't cache this error
return StatementMetadata{}, err
}
tableMetadata, found := keyspaceMetadata.Tables[meta.Table]
if !found {
// unlikely that the statement could be prepared and the metadata for
// the table couldn't be found, but this may indicate either a bug
// in the metadata code, or that the table was just dropped.
return StatementMetadata{}, ErrNoMetadata
}
meta.PKBindColumnIndexes = make([]int, len(tableMetadata.PartitionKey))
for keyIndex, keyColumn := range tableMetadata.PartitionKey {
// set an indicator for checking if the mapping is missing
meta.PKBindColumnIndexes[keyIndex] = -1
// find the column in the query info
for colIndex, boundColumn := range info.request.columns {
if keyColumn.Name == boundColumn.Name {
// there may be many such bound columns, pick the first
meta.PKBindColumnIndexes[keyIndex] = colIndex
break
}
}
if meta.PKBindColumnIndexes[keyIndex] == -1 {
// the partition key column is not bound to the statement
meta.PKBindColumnIndexes = nil
break
}
}
}
return meta, nil
}
// Exec executes a batch operation and returns nil if successful
// otherwise an error is returned describing the failure.
func (b *Batch) Exec() error {
iter := b.session.executeBatch(b, b.context)
return iter.Close()
}
// ExecContext executes a batch operation with the provided context and returns nil if successful
// otherwise an error is returned describing the failure.
func (b *Batch) ExecContext(ctx context.Context) error {
iter := b.session.executeBatch(b, ctx)
return iter.Close()
}
// Iter executes a batch operation and returns an Iter object
// that can be used to access properties related to the execution like Iter.Attempts and Iter.Latency
func (b *Batch) Iter() *Iter { return b.IterContext(b.context) }
// IterContext executes a batch operation with the provided context and returns an Iter object
// that can be used to access properties related to the execution like Iter.Attempts and Iter.Latency
func (b *Batch) IterContext(ctx context.Context) *Iter {
return b.session.executeBatch(b, ctx)
}
func (s *Session) executeBatch(batch *Batch, ctx context.Context) *Iter {
b := newInternalBatch(batch, ctx)
// fail fast
if s.Closed() {
return newErrIter(ErrSessionClosed, b.metrics, b.Keyspace(), b.getRoutingInfo(), b.getKeyspaceFunc())
}
// Prevent the execution of the batch if greater than the limit
// Currently batches have a limit of 65536 queries.
// https://datastax-oss.atlassian.net/browse/JAVA-229
if batch.Size() > BatchSizeMaximum {
return newErrIter(ErrTooManyStmts, b.metrics, b.Keyspace(), b.getRoutingInfo(), b.getKeyspaceFunc())
}
iter, err := s.executor.executeQuery(b)
if err != nil {
return newErrIter(err, b.metrics, b.Keyspace(), b.getRoutingInfo(), b.getKeyspaceFunc())
}
return iter
}
// Deprecated: use Batch.Exec instead.
// ExecuteBatch executes a batch operation and returns nil if successful
// otherwise an error is returned describing the failure.
func (s *Session) ExecuteBatch(batch *Batch) error {
iter := s.executeBatch(batch, batch.context)
return iter.Close()
}
// Deprecated: use Batch.ExecCAS instead
// ExecuteBatchCAS executes a batch operation and returns true if successful and
// an iterator (to scan additional rows if more than one conditional statement)
// was sent.
// Further scans on the interator must also remember to include
// the applied boolean as the first argument to *Iter.Scan
func (s *Session) ExecuteBatchCAS(batch *Batch, dest ...interface{}) (applied bool, iter *Iter, err error) {
return batch.ExecCAS(dest...)
}
// ExecCAS executes a batch operation and returns true if successful and
// an iterator (to scan additional rows if more than one conditional statement)
// was sent.
// Further scans on the interator must also remember to include
// the applied boolean as the first argument to *Iter.Scan
func (b *Batch) ExecCAS(dest ...interface{}) (applied bool, iter *Iter, err error) {
return b.ExecCASContext(b.context, dest...)
}
// ExecCASContext executes a batch operation with the provided context and returns true if successful and
// an iterator (to scan additional rows if more than one conditional statement)
// was sent.
// Further scans on the interator must also remember to include
// the applied boolean as the first argument to *Iter.Scan
func (b *Batch) ExecCASContext(ctx context.Context, dest ...interface{}) (applied bool, iter *Iter, err error) {
iter = b.session.executeBatch(b, ctx)
if err := iter.checkErrAndNotFound(); err != nil {
iter.Close()
return false, nil, err
}
if len(iter.Columns()) > 1 {
dest = append([]interface{}{&applied}, dest...)
iter.Scan(dest...)
} else {
iter.Scan(&applied)
}
return applied, iter, iter.err
}
// Deprecated: use Batch.MapExecCAS instead
// MapExecuteBatchCAS executes a batch operation much like ExecuteBatchCAS,
// however it accepts a map rather than a list of arguments for the initial
// scan.
func (s *Session) MapExecuteBatchCAS(batch *Batch, dest map[string]interface{}) (applied bool, iter *Iter, err error) {
return batch.MapExecCAS(dest)
}
// MapExecCAS executes a batch operation much like ExecuteBatchCAS,
// however it accepts a map rather than a list of arguments for the initial
// scan.
func (b *Batch) MapExecCAS(dest map[string]interface{}) (applied bool, iter *Iter, err error) {
return b.MapExecCASContext(b.context, dest)
}
// MapExecCASContext executes a batch operation with the provided context much like ExecuteBatchCAS,
// however it accepts a map rather than a list of arguments for the initial
// scan.
func (b *Batch) MapExecCASContext(ctx context.Context, dest map[string]interface{}) (applied bool, iter *Iter, err error) {
iter = b.session.executeBatch(b, ctx)
if err := iter.checkErrAndNotFound(); err != nil {
iter.Close()
return false, nil, err
}
iter.MapScan(dest)
if iter.err != nil {
return false, iter, iter.err
}
// check if [applied] was returned, otherwise it might not be CAS
if _, ok := dest["[applied]"]; ok {
applied = dest["[applied]"].(bool)
delete(dest, "[applied]")
}
// we usually close here, but instead of closing, just returin an error
// if MapScan failed. Although Close just returns err, using Close
// here might be confusing as we are not actually closing the iter
return applied, iter, iter.err
}
type hostMetrics struct {
// Attempts is count of how many times this query has been attempted for this host.
// An attempt is either a retry or fetching next page of results.
Attempts int
// TotalLatency is the sum of attempt latencies for this host in nanoseconds.
TotalLatency int64
}
type queryMetrics struct {
totalAttempts int64
totalLatency int64
}
func (qm *queryMetrics) attempt(addLatency time.Duration) int {
atomic.AddInt64(&qm.totalLatency, addLatency.Nanoseconds())
return int(atomic.AddInt64(&qm.totalAttempts, 1) - 1)
}
func (qm *queryMetrics) attempts() int {
return int(atomic.LoadInt64(&qm.totalAttempts))
}
func (qm *queryMetrics) latency() int64 {
attempts := atomic.LoadInt64(&qm.totalAttempts)
if attempts == 0 {
return atomic.LoadInt64(&qm.totalLatency)
}
return atomic.LoadInt64(&qm.totalLatency) / attempts
}
type hostMetricsManager interface {
attempt(addLatency time.Duration, host *HostInfo) *hostMetrics
}
type hostMetricsManagerImpl struct {
l sync.RWMutex
m map[string]*hostMetrics
}