session.go 49 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806
  1. // Copyright (c) 2012 The gocql Authors. All rights reserved.
  2. // Use of this source code is governed by a BSD-style
  3. // license that can be found in the LICENSE file.
  4. package gocql
  5. import (
  6. "bytes"
  7. "context"
  8. "encoding/binary"
  9. "errors"
  10. "fmt"
  11. "io"
  12. "net"
  13. "strings"
  14. "sync"
  15. "sync/atomic"
  16. "time"
  17. "unicode"
  18. "github.com/gocql/gocql/internal/lru"
  19. )
  20. // Session is the interface used by users to interact with the database.
  21. //
  22. // It's safe for concurrent use by multiple goroutines and a typical usage
  23. // scenario is to have one global session object to interact with the
  24. // whole Cassandra cluster.
  25. //
  26. // This type extends the Node interface by adding a convinient query builder
  27. // and automatically sets a default consistency level on all operations
  28. // that do not have a consistency level set.
  29. type Session struct {
  30. cons Consistency
  31. pageSize int
  32. prefetch float64
  33. routingKeyInfoCache routingKeyInfoLRU
  34. schemaDescriber *schemaDescriber
  35. trace Tracer
  36. queryObserver QueryObserver
  37. batchObserver BatchObserver
  38. connectObserver ConnectObserver
  39. frameObserver FrameHeaderObserver
  40. hostSource *ringDescriber
  41. stmtsLRU *preparedLRU
  42. connCfg *ConnConfig
  43. executor *queryExecutor
  44. pool *policyConnPool
  45. policy HostSelectionPolicy
  46. ring ring
  47. metadata clusterMetadata
  48. mu sync.RWMutex
  49. control *controlConn
  50. // event handlers
  51. nodeEvents *eventDebouncer
  52. schemaEvents *eventDebouncer
  53. // ring metadata
  54. hosts []HostInfo
  55. useSystemSchema bool
  56. cfg ClusterConfig
  57. quit chan struct{}
  58. closeMu sync.RWMutex
  59. isClosed bool
  60. }
  61. var queryPool = &sync.Pool{
  62. New: func() interface{} {
  63. return new(Query)
  64. },
  65. }
  66. func addrsToHosts(addrs []string, defaultPort int) ([]*HostInfo, error) {
  67. var hosts []*HostInfo
  68. for _, hostport := range addrs {
  69. resolvedHosts, err := hostInfo(hostport, defaultPort)
  70. if err != nil {
  71. // Try other hosts if unable to resolve DNS name
  72. if _, ok := err.(*net.DNSError); ok {
  73. Logger.Printf("gocql: dns error: %v\n", err)
  74. continue
  75. }
  76. return nil, err
  77. }
  78. hosts = append(hosts, resolvedHosts...)
  79. }
  80. if len(hosts) == 0 {
  81. return nil, errors.New("failed to resolve any of the provided hostnames")
  82. }
  83. return hosts, nil
  84. }
  85. // NewSession wraps an existing Node.
  86. func NewSession(cfg ClusterConfig) (*Session, error) {
  87. // Check that hosts in the ClusterConfig is not empty
  88. if len(cfg.Hosts) < 1 {
  89. return nil, ErrNoHosts
  90. }
  91. s := &Session{
  92. cons: cfg.Consistency,
  93. prefetch: 0.25,
  94. cfg: cfg,
  95. pageSize: cfg.PageSize,
  96. stmtsLRU: &preparedLRU{lru: lru.New(cfg.MaxPreparedStmts)},
  97. quit: make(chan struct{}),
  98. connectObserver: cfg.ConnectObserver,
  99. }
  100. s.schemaDescriber = newSchemaDescriber(s)
  101. s.nodeEvents = newEventDebouncer("NodeEvents", s.handleNodeEvent)
  102. s.schemaEvents = newEventDebouncer("SchemaEvents", s.handleSchemaEvent)
  103. s.routingKeyInfoCache.lru = lru.New(cfg.MaxRoutingKeyInfo)
  104. s.hostSource = &ringDescriber{session: s}
  105. if cfg.PoolConfig.HostSelectionPolicy == nil {
  106. cfg.PoolConfig.HostSelectionPolicy = RoundRobinHostPolicy()
  107. }
  108. s.pool = cfg.PoolConfig.buildPool(s)
  109. s.policy = cfg.PoolConfig.HostSelectionPolicy
  110. s.policy.Init(s)
  111. s.executor = &queryExecutor{
  112. pool: s.pool,
  113. policy: cfg.PoolConfig.HostSelectionPolicy,
  114. }
  115. s.queryObserver = cfg.QueryObserver
  116. s.batchObserver = cfg.BatchObserver
  117. s.connectObserver = cfg.ConnectObserver
  118. s.frameObserver = cfg.FrameHeaderObserver
  119. //Check the TLS Config before trying to connect to anything external
  120. connCfg, err := connConfig(&s.cfg)
  121. if err != nil {
  122. //TODO: Return a typed error
  123. return nil, fmt.Errorf("gocql: unable to create session: %v", err)
  124. }
  125. s.connCfg = connCfg
  126. if err := s.init(); err != nil {
  127. s.Close()
  128. if err == ErrNoConnectionsStarted {
  129. //This error used to be generated inside NewSession & returned directly
  130. //Forward it on up to be backwards compatible
  131. return nil, ErrNoConnectionsStarted
  132. } else {
  133. // TODO(zariel): dont wrap this error in fmt.Errorf, return a typed error
  134. return nil, fmt.Errorf("gocql: unable to create session: %v", err)
  135. }
  136. }
  137. return s, nil
  138. }
  139. func (s *Session) init() error {
  140. hosts, err := addrsToHosts(s.cfg.Hosts, s.cfg.Port)
  141. if err != nil {
  142. return err
  143. }
  144. s.ring.endpoints = hosts
  145. if !s.cfg.disableControlConn {
  146. s.control = createControlConn(s)
  147. if s.cfg.ProtoVersion == 0 {
  148. proto, err := s.control.discoverProtocol(hosts)
  149. if err != nil {
  150. return fmt.Errorf("unable to discover protocol version: %v", err)
  151. } else if proto == 0 {
  152. return errors.New("unable to discovery protocol version")
  153. }
  154. // TODO(zariel): we really only need this in 1 place
  155. s.cfg.ProtoVersion = proto
  156. s.connCfg.ProtoVersion = proto
  157. }
  158. if err := s.control.connect(hosts); err != nil {
  159. return err
  160. }
  161. if !s.cfg.DisableInitialHostLookup {
  162. var partitioner string
  163. newHosts, partitioner, err := s.hostSource.GetHosts()
  164. if err != nil {
  165. return err
  166. }
  167. s.policy.SetPartitioner(partitioner)
  168. filteredHosts := make([]*HostInfo, 0, len(newHosts))
  169. for _, host := range newHosts {
  170. if !s.cfg.filterHost(host) {
  171. filteredHosts = append(filteredHosts, host)
  172. }
  173. }
  174. hosts = append(hosts, filteredHosts...)
  175. }
  176. }
  177. hostMap := make(map[string]*HostInfo, len(hosts))
  178. for _, host := range hosts {
  179. hostMap[host.ConnectAddress().String()] = host
  180. }
  181. for _, host := range hostMap {
  182. host = s.ring.addOrUpdate(host)
  183. s.addNewNode(host)
  184. }
  185. // TODO(zariel): we probably dont need this any more as we verify that we
  186. // can connect to one of the endpoints supplied by using the control conn.
  187. // See if there are any connections in the pool
  188. if s.cfg.ReconnectInterval > 0 {
  189. go s.reconnectDownedHosts(s.cfg.ReconnectInterval)
  190. }
  191. // If we disable the initial host lookup, we need to still check if the
  192. // cluster is using the newer system schema or not... however, if control
  193. // connection is disable, we really have no choice, so we just make our
  194. // best guess...
  195. if !s.cfg.disableControlConn && s.cfg.DisableInitialHostLookup {
  196. newer, _ := checkSystemSchema(s.control)
  197. s.useSystemSchema = newer
  198. } else {
  199. host := s.ring.rrHost()
  200. s.useSystemSchema = host.Version().Major >= 3
  201. }
  202. if s.pool.Size() == 0 {
  203. return ErrNoConnectionsStarted
  204. }
  205. return nil
  206. }
  207. func (s *Session) reconnectDownedHosts(intv time.Duration) {
  208. reconnectTicker := time.NewTicker(intv)
  209. defer reconnectTicker.Stop()
  210. for {
  211. select {
  212. case <-reconnectTicker.C:
  213. hosts := s.ring.allHosts()
  214. // Print session.ring for debug.
  215. if gocqlDebug {
  216. buf := bytes.NewBufferString("Session.ring:")
  217. for _, h := range hosts {
  218. buf.WriteString("[" + h.ConnectAddress().String() + ":" + h.State().String() + "]")
  219. }
  220. Logger.Println(buf.String())
  221. }
  222. for _, h := range hosts {
  223. if h.IsUp() {
  224. continue
  225. }
  226. s.handleNodeUp(h.ConnectAddress(), h.Port(), true)
  227. }
  228. case <-s.quit:
  229. return
  230. }
  231. }
  232. }
  233. // SetConsistency sets the default consistency level for this session. This
  234. // setting can also be changed on a per-query basis and the default value
  235. // is Quorum.
  236. func (s *Session) SetConsistency(cons Consistency) {
  237. s.mu.Lock()
  238. s.cons = cons
  239. s.mu.Unlock()
  240. }
  241. // SetPageSize sets the default page size for this session. A value <= 0 will
  242. // disable paging. This setting can also be changed on a per-query basis.
  243. func (s *Session) SetPageSize(n int) {
  244. s.mu.Lock()
  245. s.pageSize = n
  246. s.mu.Unlock()
  247. }
  248. // SetPrefetch sets the default threshold for pre-fetching new pages. If
  249. // there are only p*pageSize rows remaining, the next page will be requested
  250. // automatically. This value can also be changed on a per-query basis and
  251. // the default value is 0.25.
  252. func (s *Session) SetPrefetch(p float64) {
  253. s.mu.Lock()
  254. s.prefetch = p
  255. s.mu.Unlock()
  256. }
  257. // SetTrace sets the default tracer for this session. This setting can also
  258. // be changed on a per-query basis.
  259. func (s *Session) SetTrace(trace Tracer) {
  260. s.mu.Lock()
  261. s.trace = trace
  262. s.mu.Unlock()
  263. }
  264. // Query generates a new query object for interacting with the database.
  265. // Further details of the query may be tweaked using the resulting query
  266. // value before the query is executed. Query is automatically prepared
  267. // if it has not previously been executed.
  268. func (s *Session) Query(stmt string, values ...interface{}) *Query {
  269. qry := queryPool.Get().(*Query)
  270. qry.session = s
  271. qry.stmt = stmt
  272. qry.values = values
  273. qry.defaultsFromSession()
  274. return qry
  275. }
  276. type QueryInfo struct {
  277. Id []byte
  278. Args []ColumnInfo
  279. Rval []ColumnInfo
  280. PKeyColumns []int
  281. }
  282. // Bind generates a new query object based on the query statement passed in.
  283. // The query is automatically prepared if it has not previously been executed.
  284. // The binding callback allows the application to define which query argument
  285. // values will be marshalled as part of the query execution.
  286. // During execution, the meta data of the prepared query will be routed to the
  287. // binding callback, which is responsible for producing the query argument values.
  288. func (s *Session) Bind(stmt string, b func(q *QueryInfo) ([]interface{}, error)) *Query {
  289. qry := queryPool.Get().(*Query)
  290. qry.session = s
  291. qry.stmt = stmt
  292. qry.binding = b
  293. qry.defaultsFromSession()
  294. return qry
  295. }
  296. // Close closes all connections. The session is unusable after this
  297. // operation.
  298. func (s *Session) Close() {
  299. s.closeMu.Lock()
  300. defer s.closeMu.Unlock()
  301. if s.isClosed {
  302. return
  303. }
  304. s.isClosed = true
  305. if s.pool != nil {
  306. s.pool.Close()
  307. }
  308. if s.control != nil {
  309. s.control.close()
  310. }
  311. if s.nodeEvents != nil {
  312. s.nodeEvents.stop()
  313. }
  314. if s.schemaEvents != nil {
  315. s.schemaEvents.stop()
  316. }
  317. if s.quit != nil {
  318. close(s.quit)
  319. }
  320. }
  321. func (s *Session) Closed() bool {
  322. s.closeMu.RLock()
  323. closed := s.isClosed
  324. s.closeMu.RUnlock()
  325. return closed
  326. }
  327. func (s *Session) executeQuery(qry *Query) (it *Iter) {
  328. // fail fast
  329. if s.Closed() {
  330. return &Iter{err: ErrSessionClosed}
  331. }
  332. iter, err := s.executor.executeQuery(qry)
  333. if err != nil {
  334. return &Iter{err: err}
  335. }
  336. if iter == nil {
  337. panic("nil iter")
  338. }
  339. return iter
  340. }
  341. func (s *Session) removeHost(h *HostInfo) {
  342. s.policy.RemoveHost(h)
  343. s.pool.removeHost(h.ConnectAddress())
  344. s.ring.removeHost(h.ConnectAddress())
  345. }
  346. // KeyspaceMetadata returns the schema metadata for the keyspace specified. Returns an error if the keyspace does not exist.
  347. func (s *Session) KeyspaceMetadata(keyspace string) (*KeyspaceMetadata, error) {
  348. // fail fast
  349. if s.Closed() {
  350. return nil, ErrSessionClosed
  351. } else if keyspace == "" {
  352. return nil, ErrNoKeyspace
  353. }
  354. return s.schemaDescriber.getSchema(keyspace)
  355. }
  356. func (s *Session) getConn() *Conn {
  357. hosts := s.ring.allHosts()
  358. for _, host := range hosts {
  359. if !host.IsUp() {
  360. continue
  361. }
  362. pool, ok := s.pool.getPool(host)
  363. if !ok {
  364. continue
  365. } else if conn := pool.Pick(); conn != nil {
  366. return conn
  367. }
  368. }
  369. return nil
  370. }
  371. // returns routing key indexes and type info
  372. func (s *Session) routingKeyInfo(ctx context.Context, stmt string) (*routingKeyInfo, error) {
  373. s.routingKeyInfoCache.mu.Lock()
  374. entry, cached := s.routingKeyInfoCache.lru.Get(stmt)
  375. if cached {
  376. // done accessing the cache
  377. s.routingKeyInfoCache.mu.Unlock()
  378. // the entry is an inflight struct similar to that used by
  379. // Conn to prepare statements
  380. inflight := entry.(*inflightCachedEntry)
  381. // wait for any inflight work
  382. inflight.wg.Wait()
  383. if inflight.err != nil {
  384. return nil, inflight.err
  385. }
  386. key, _ := inflight.value.(*routingKeyInfo)
  387. return key, nil
  388. }
  389. // create a new inflight entry while the data is created
  390. inflight := new(inflightCachedEntry)
  391. inflight.wg.Add(1)
  392. defer inflight.wg.Done()
  393. s.routingKeyInfoCache.lru.Add(stmt, inflight)
  394. s.routingKeyInfoCache.mu.Unlock()
  395. var (
  396. info *preparedStatment
  397. partitionKey []*ColumnMetadata
  398. )
  399. conn := s.getConn()
  400. if conn == nil {
  401. // TODO: better error?
  402. inflight.err = errors.New("gocql: unable to fetch prepared info: no connection available")
  403. return nil, inflight.err
  404. }
  405. // get the query info for the statement
  406. info, inflight.err = conn.prepareStatement(ctx, stmt, nil)
  407. if inflight.err != nil {
  408. // don't cache this error
  409. s.routingKeyInfoCache.Remove(stmt)
  410. return nil, inflight.err
  411. }
  412. // TODO: it would be nice to mark hosts here but as we are not using the policies
  413. // to fetch hosts we cant
  414. if info.request.colCount == 0 {
  415. // no arguments, no routing key, and no error
  416. return nil, nil
  417. }
  418. if len(info.request.pkeyColumns) > 0 {
  419. // proto v4 dont need to calculate primary key columns
  420. types := make([]TypeInfo, len(info.request.pkeyColumns))
  421. for i, col := range info.request.pkeyColumns {
  422. types[i] = info.request.columns[col].TypeInfo
  423. }
  424. routingKeyInfo := &routingKeyInfo{
  425. indexes: info.request.pkeyColumns,
  426. types: types,
  427. }
  428. inflight.value = routingKeyInfo
  429. return routingKeyInfo, nil
  430. }
  431. // get the table metadata
  432. table := info.request.columns[0].Table
  433. var keyspaceMetadata *KeyspaceMetadata
  434. keyspaceMetadata, inflight.err = s.KeyspaceMetadata(info.request.columns[0].Keyspace)
  435. if inflight.err != nil {
  436. // don't cache this error
  437. s.routingKeyInfoCache.Remove(stmt)
  438. return nil, inflight.err
  439. }
  440. tableMetadata, found := keyspaceMetadata.Tables[table]
  441. if !found {
  442. // unlikely that the statement could be prepared and the metadata for
  443. // the table couldn't be found, but this may indicate either a bug
  444. // in the metadata code, or that the table was just dropped.
  445. inflight.err = ErrNoMetadata
  446. // don't cache this error
  447. s.routingKeyInfoCache.Remove(stmt)
  448. return nil, inflight.err
  449. }
  450. partitionKey = tableMetadata.PartitionKey
  451. size := len(partitionKey)
  452. routingKeyInfo := &routingKeyInfo{
  453. indexes: make([]int, size),
  454. types: make([]TypeInfo, size),
  455. }
  456. for keyIndex, keyColumn := range partitionKey {
  457. // set an indicator for checking if the mapping is missing
  458. routingKeyInfo.indexes[keyIndex] = -1
  459. // find the column in the query info
  460. for argIndex, boundColumn := range info.request.columns {
  461. if keyColumn.Name == boundColumn.Name {
  462. // there may be many such bound columns, pick the first
  463. routingKeyInfo.indexes[keyIndex] = argIndex
  464. routingKeyInfo.types[keyIndex] = boundColumn.TypeInfo
  465. break
  466. }
  467. }
  468. if routingKeyInfo.indexes[keyIndex] == -1 {
  469. // missing a routing key column mapping
  470. // no routing key, and no error
  471. return nil, nil
  472. }
  473. }
  474. // cache this result
  475. inflight.value = routingKeyInfo
  476. return routingKeyInfo, nil
  477. }
  478. func (b *Batch) execute(conn *Conn) *Iter {
  479. return conn.executeBatch(b)
  480. }
  481. func (s *Session) executeBatch(batch *Batch) *Iter {
  482. // fail fast
  483. if s.Closed() {
  484. return &Iter{err: ErrSessionClosed}
  485. }
  486. // Prevent the execution of the batch if greater than the limit
  487. // Currently batches have a limit of 65536 queries.
  488. // https://datastax-oss.atlassian.net/browse/JAVA-229
  489. if batch.Size() > BatchSizeMaximum {
  490. return &Iter{err: ErrTooManyStmts}
  491. }
  492. iter, err := s.executor.executeQuery(batch)
  493. if err != nil {
  494. return &Iter{err: err}
  495. }
  496. return iter
  497. }
  498. // ExecuteBatch executes a batch operation and returns nil if successful
  499. // otherwise an error is returned describing the failure.
  500. func (s *Session) ExecuteBatch(batch *Batch) error {
  501. iter := s.executeBatch(batch)
  502. return iter.Close()
  503. }
  504. // ExecuteBatchCAS executes a batch operation and returns true if successful and
  505. // an iterator (to scan aditional rows if more than one conditional statement)
  506. // was sent.
  507. // Further scans on the interator must also remember to include
  508. // the applied boolean as the first argument to *Iter.Scan
  509. func (s *Session) ExecuteBatchCAS(batch *Batch, dest ...interface{}) (applied bool, iter *Iter, err error) {
  510. iter = s.executeBatch(batch)
  511. if err := iter.checkErrAndNotFound(); err != nil {
  512. iter.Close()
  513. return false, nil, err
  514. }
  515. if len(iter.Columns()) > 1 {
  516. dest = append([]interface{}{&applied}, dest...)
  517. iter.Scan(dest...)
  518. } else {
  519. iter.Scan(&applied)
  520. }
  521. return applied, iter, nil
  522. }
  523. // MapExecuteBatchCAS executes a batch operation much like ExecuteBatchCAS,
  524. // however it accepts a map rather than a list of arguments for the initial
  525. // scan.
  526. func (s *Session) MapExecuteBatchCAS(batch *Batch, dest map[string]interface{}) (applied bool, iter *Iter, err error) {
  527. iter = s.executeBatch(batch)
  528. if err := iter.checkErrAndNotFound(); err != nil {
  529. iter.Close()
  530. return false, nil, err
  531. }
  532. iter.MapScan(dest)
  533. applied = dest["[applied]"].(bool)
  534. delete(dest, "[applied]")
  535. // we usually close here, but instead of closing, just returin an error
  536. // if MapScan failed. Although Close just returns err, using Close
  537. // here might be confusing as we are not actually closing the iter
  538. return applied, iter, iter.err
  539. }
  540. func (s *Session) connect(host *HostInfo, errorHandler ConnErrorHandler) (*Conn, error) {
  541. if s.connectObserver != nil {
  542. obs := ObservedConnect{
  543. Host: host,
  544. Start: time.Now(),
  545. }
  546. conn, err := s.dial(host, s.connCfg, errorHandler)
  547. obs.End = time.Now()
  548. obs.Err = err
  549. s.connectObserver.ObserveConnect(obs)
  550. return conn, err
  551. }
  552. return s.dial(host, s.connCfg, errorHandler)
  553. }
  554. // Query represents a CQL statement that can be executed.
  555. type Query struct {
  556. stmt string
  557. values []interface{}
  558. cons Consistency
  559. pageSize int
  560. routingKey []byte
  561. routingKeyBuffer []byte
  562. pageState []byte
  563. prefetch float64
  564. trace Tracer
  565. observer QueryObserver
  566. session *Session
  567. rt RetryPolicy
  568. binding func(q *QueryInfo) ([]interface{}, error)
  569. attempts int
  570. totalLatency int64
  571. serialCons SerialConsistency
  572. defaultTimestamp bool
  573. defaultTimestampValue int64
  574. disableSkipMetadata bool
  575. context context.Context
  576. idempotent bool
  577. attemptTimeoutTimer *time.Timer
  578. disableAutoPage bool
  579. }
  580. func (q *Query) defaultsFromSession() {
  581. s := q.session
  582. s.mu.RLock()
  583. q.cons = s.cons
  584. q.pageSize = s.pageSize
  585. q.trace = s.trace
  586. q.observer = s.queryObserver
  587. q.prefetch = s.prefetch
  588. q.rt = s.cfg.RetryPolicy
  589. q.serialCons = s.cfg.SerialConsistency
  590. q.defaultTimestamp = s.cfg.DefaultTimestamp
  591. q.idempotent = s.cfg.DefaultIdempotence
  592. s.mu.RUnlock()
  593. }
  594. // Statement returns the statement that was used to generate this query.
  595. func (q Query) Statement() string {
  596. return q.stmt
  597. }
  598. // String implements the stringer interface.
  599. func (q Query) String() string {
  600. return fmt.Sprintf("[query statement=%q values=%+v consistency=%s]", q.stmt, q.values, q.cons)
  601. }
  602. //Attempts returns the number of times the query was executed.
  603. func (q *Query) Attempts() int {
  604. return q.attempts
  605. }
  606. //Latency returns the average amount of nanoseconds per attempt of the query.
  607. func (q *Query) Latency() int64 {
  608. if q.attempts > 0 {
  609. return q.totalLatency / int64(q.attempts)
  610. }
  611. return 0
  612. }
  613. // Consistency sets the consistency level for this query. If no consistency
  614. // level have been set, the default consistency level of the cluster
  615. // is used.
  616. func (q *Query) Consistency(c Consistency) *Query {
  617. q.cons = c
  618. return q
  619. }
  620. // GetConsistency returns the currently configured consistency level for
  621. // the query.
  622. func (q *Query) GetConsistency() Consistency {
  623. return q.cons
  624. }
  625. // Same as Consistency but without a return value
  626. func (q *Query) SetConsistency(c Consistency) {
  627. q.cons = c
  628. }
  629. // Trace enables tracing of this query. Look at the documentation of the
  630. // Tracer interface to learn more about tracing.
  631. func (q *Query) Trace(trace Tracer) *Query {
  632. q.trace = trace
  633. return q
  634. }
  635. // Observer enables query-level observer on this query.
  636. // The provided observer will be called every time this query is executed.
  637. func (q *Query) Observer(observer QueryObserver) *Query {
  638. q.observer = observer
  639. return q
  640. }
  641. // PageSize will tell the iterator to fetch the result in pages of size n.
  642. // This is useful for iterating over large result sets, but setting the
  643. // page size too low might decrease the performance. This feature is only
  644. // available in Cassandra 2 and onwards.
  645. func (q *Query) PageSize(n int) *Query {
  646. q.pageSize = n
  647. return q
  648. }
  649. // DefaultTimestamp will enable the with default timestamp flag on the query.
  650. // If enable, this will replace the server side assigned
  651. // timestamp as default timestamp. Note that a timestamp in the query itself
  652. // will still override this timestamp. This is entirely optional.
  653. //
  654. // Only available on protocol >= 3
  655. func (q *Query) DefaultTimestamp(enable bool) *Query {
  656. q.defaultTimestamp = enable
  657. return q
  658. }
  659. // WithTimestamp will enable the with default timestamp flag on the query
  660. // like DefaultTimestamp does. But also allows to define value for timestamp.
  661. // It works the same way as USING TIMESTAMP in the query itself, but
  662. // should not break prepared query optimization
  663. //
  664. // Only available on protocol >= 3
  665. func (q *Query) WithTimestamp(timestamp int64) *Query {
  666. q.DefaultTimestamp(true)
  667. q.defaultTimestampValue = timestamp
  668. return q
  669. }
  670. // RoutingKey sets the routing key to use when a token aware connection
  671. // pool is used to optimize the routing of this query.
  672. func (q *Query) RoutingKey(routingKey []byte) *Query {
  673. q.routingKey = routingKey
  674. return q
  675. }
  676. // WithContext will set the context to use during a query, it will be used to
  677. // timeout when waiting for responses from Cassandra.
  678. func (q *Query) WithContext(ctx context.Context) *Query {
  679. q.context = ctx
  680. return q
  681. }
  682. // Context satisfies the ExecutableQuery interface.
  683. func (q *Query) Context() context.Context {
  684. return q.context
  685. }
  686. func (q *Query) execute(conn *Conn) *Iter {
  687. return conn.executeQuery(q)
  688. }
  689. func (q *Query) attempt(keyspace string, end, start time.Time, iter *Iter, host *HostInfo) {
  690. q.attempts++
  691. q.totalLatency += end.Sub(start).Nanoseconds()
  692. // TODO: track latencies per host and things as well instead of just total
  693. if q.observer != nil {
  694. q.observer.ObserveQuery(q.context, ObservedQuery{
  695. Keyspace: keyspace,
  696. Statement: q.stmt,
  697. Start: start,
  698. End: end,
  699. Rows: iter.numRows,
  700. Host: host,
  701. Err: iter.err,
  702. Attempt: q.attempts,
  703. })
  704. }
  705. }
  706. func (q *Query) retryPolicy() RetryPolicy {
  707. return q.rt
  708. }
  709. // Keyspace returns the keyspace the query will be executed against.
  710. func (q *Query) Keyspace() string {
  711. if q.session == nil {
  712. return ""
  713. }
  714. // TODO(chbannis): this should be parsed from the query or we should let
  715. // this be set by users.
  716. return q.session.cfg.Keyspace
  717. }
  718. // GetRoutingKey gets the routing key to use for routing this query. If
  719. // a routing key has not been explicitly set, then the routing key will
  720. // be constructed if possible using the keyspace's schema and the query
  721. // info for this query statement. If the routing key cannot be determined
  722. // then nil will be returned with no error. On any error condition,
  723. // an error description will be returned.
  724. func (q *Query) GetRoutingKey() ([]byte, error) {
  725. if q.routingKey != nil {
  726. return q.routingKey, nil
  727. } else if q.binding != nil && len(q.values) == 0 {
  728. // If this query was created using session.Bind we wont have the query
  729. // values yet, so we have to pass down to the next policy.
  730. // TODO: Remove this and handle this case
  731. return nil, nil
  732. }
  733. // try to determine the routing key
  734. routingKeyInfo, err := q.session.routingKeyInfo(q.context, q.stmt)
  735. if err != nil {
  736. return nil, err
  737. }
  738. if routingKeyInfo == nil {
  739. return nil, nil
  740. }
  741. if len(routingKeyInfo.indexes) == 1 {
  742. // single column routing key
  743. routingKey, err := Marshal(
  744. routingKeyInfo.types[0],
  745. q.values[routingKeyInfo.indexes[0]],
  746. )
  747. if err != nil {
  748. return nil, err
  749. }
  750. return routingKey, nil
  751. }
  752. // We allocate that buffer only once, so that further re-bind/exec of the
  753. // same query don't allocate more memory.
  754. if q.routingKeyBuffer == nil {
  755. q.routingKeyBuffer = make([]byte, 0, 256)
  756. }
  757. // composite routing key
  758. buf := bytes.NewBuffer(q.routingKeyBuffer)
  759. for i := range routingKeyInfo.indexes {
  760. encoded, err := Marshal(
  761. routingKeyInfo.types[i],
  762. q.values[routingKeyInfo.indexes[i]],
  763. )
  764. if err != nil {
  765. return nil, err
  766. }
  767. lenBuf := []byte{0x00, 0x00}
  768. binary.BigEndian.PutUint16(lenBuf, uint16(len(encoded)))
  769. buf.Write(lenBuf)
  770. buf.Write(encoded)
  771. buf.WriteByte(0x00)
  772. }
  773. routingKey := buf.Bytes()
  774. return routingKey, nil
  775. }
  776. func (q *Query) shouldPrepare() bool {
  777. stmt := strings.TrimLeftFunc(strings.TrimRightFunc(q.stmt, func(r rune) bool {
  778. return unicode.IsSpace(r) || r == ';'
  779. }), unicode.IsSpace)
  780. var stmtType string
  781. if n := strings.IndexFunc(stmt, unicode.IsSpace); n >= 0 {
  782. stmtType = strings.ToLower(stmt[:n])
  783. }
  784. if stmtType == "begin" {
  785. if n := strings.LastIndexFunc(stmt, unicode.IsSpace); n >= 0 {
  786. stmtType = strings.ToLower(stmt[n+1:])
  787. }
  788. }
  789. switch stmtType {
  790. case "select", "insert", "update", "delete", "batch":
  791. return true
  792. }
  793. return false
  794. }
  795. // SetPrefetch sets the default threshold for pre-fetching new pages. If
  796. // there are only p*pageSize rows remaining, the next page will be requested
  797. // automatically.
  798. func (q *Query) Prefetch(p float64) *Query {
  799. q.prefetch = p
  800. return q
  801. }
  802. // RetryPolicy sets the policy to use when retrying the query.
  803. func (q *Query) RetryPolicy(r RetryPolicy) *Query {
  804. q.rt = r
  805. return q
  806. }
  807. func (q *Query) IsIdempotent() bool {
  808. return q.idempotent
  809. }
  810. // Idempotent marks the query as being idempotent or not depending on
  811. // the value.
  812. func (q *Query) Idempotent(value bool) *Query {
  813. q.idempotent = value
  814. return q
  815. }
  816. // Bind sets query arguments of query. This can also be used to rebind new query arguments
  817. // to an existing query instance.
  818. func (q *Query) Bind(v ...interface{}) *Query {
  819. q.values = v
  820. return q
  821. }
  822. // SerialConsistency sets the consistency level for the
  823. // serial phase of conditional updates. That consistency can only be
  824. // either SERIAL or LOCAL_SERIAL and if not present, it defaults to
  825. // SERIAL. This option will be ignored for anything else that a
  826. // conditional update/insert.
  827. func (q *Query) SerialConsistency(cons SerialConsistency) *Query {
  828. q.serialCons = cons
  829. return q
  830. }
  831. // PageState sets the paging state for the query to resume paging from a specific
  832. // point in time. Setting this will disable to query paging for this query, and
  833. // must be used for all subsequent pages.
  834. func (q *Query) PageState(state []byte) *Query {
  835. q.pageState = state
  836. q.disableAutoPage = true
  837. return q
  838. }
  839. // NoSkipMetadata will override the internal result metadata cache so that the driver does not
  840. // send skip_metadata for queries, this means that the result will always contain
  841. // the metadata to parse the rows and will not reuse the metadata from the prepared
  842. // staement. This should only be used to work around cassandra bugs, such as when using
  843. // CAS operations which do not end in Cas.
  844. //
  845. // See https://issues.apache.org/jira/browse/CASSANDRA-11099
  846. // https://github.com/gocql/gocql/issues/612
  847. func (q *Query) NoSkipMetadata() *Query {
  848. q.disableSkipMetadata = true
  849. return q
  850. }
  851. // Exec executes the query without returning any rows.
  852. func (q *Query) Exec() error {
  853. return q.Iter().Close()
  854. }
  855. func isUseStatement(stmt string) bool {
  856. if len(stmt) < 3 {
  857. return false
  858. }
  859. return strings.EqualFold(stmt[0:3], "use")
  860. }
  861. // Iter executes the query and returns an iterator capable of iterating
  862. // over all results.
  863. func (q *Query) Iter() *Iter {
  864. if isUseStatement(q.stmt) {
  865. return &Iter{err: ErrUseStmt}
  866. }
  867. return q.session.executeQuery(q)
  868. }
  869. // MapScan executes the query, copies the columns of the first selected
  870. // row into the map pointed at by m and discards the rest. If no rows
  871. // were selected, ErrNotFound is returned.
  872. func (q *Query) MapScan(m map[string]interface{}) error {
  873. iter := q.Iter()
  874. if err := iter.checkErrAndNotFound(); err != nil {
  875. return err
  876. }
  877. iter.MapScan(m)
  878. return iter.Close()
  879. }
  880. // Scan executes the query, copies the columns of the first selected
  881. // row into the values pointed at by dest and discards the rest. If no rows
  882. // were selected, ErrNotFound is returned.
  883. func (q *Query) Scan(dest ...interface{}) error {
  884. iter := q.Iter()
  885. if err := iter.checkErrAndNotFound(); err != nil {
  886. return err
  887. }
  888. iter.Scan(dest...)
  889. return iter.Close()
  890. }
  891. // ScanCAS executes a lightweight transaction (i.e. an UPDATE or INSERT
  892. // statement containing an IF clause). If the transaction fails because
  893. // the existing values did not match, the previous values will be stored
  894. // in dest.
  895. func (q *Query) ScanCAS(dest ...interface{}) (applied bool, err error) {
  896. q.disableSkipMetadata = true
  897. iter := q.Iter()
  898. if err := iter.checkErrAndNotFound(); err != nil {
  899. return false, err
  900. }
  901. if len(iter.Columns()) > 1 {
  902. dest = append([]interface{}{&applied}, dest...)
  903. iter.Scan(dest...)
  904. } else {
  905. iter.Scan(&applied)
  906. }
  907. return applied, iter.Close()
  908. }
  909. // MapScanCAS executes a lightweight transaction (i.e. an UPDATE or INSERT
  910. // statement containing an IF clause). If the transaction fails because
  911. // the existing values did not match, the previous values will be stored
  912. // in dest map.
  913. //
  914. // As for INSERT .. IF NOT EXISTS, previous values will be returned as if
  915. // SELECT * FROM. So using ScanCAS with INSERT is inherently prone to
  916. // column mismatching. MapScanCAS is added to capture them safely.
  917. func (q *Query) MapScanCAS(dest map[string]interface{}) (applied bool, err error) {
  918. q.disableSkipMetadata = true
  919. iter := q.Iter()
  920. if err := iter.checkErrAndNotFound(); err != nil {
  921. return false, err
  922. }
  923. iter.MapScan(dest)
  924. applied = dest["[applied]"].(bool)
  925. delete(dest, "[applied]")
  926. return applied, iter.Close()
  927. }
  928. // Release releases a query back into a pool of queries. Released Queries
  929. // cannot be reused.
  930. //
  931. // Example:
  932. // qry := session.Query("SELECT * FROM my_table")
  933. // qry.Exec()
  934. // qry.Release()
  935. func (q *Query) Release() {
  936. q.reset()
  937. queryPool.Put(q)
  938. }
  939. // reset zeroes out all fields of a query so that it can be safely pooled.
  940. func (q *Query) reset() {
  941. *q = Query{}
  942. }
  943. // Iter represents an iterator that can be used to iterate over all rows that
  944. // were returned by a query. The iterator might send additional queries to the
  945. // database during the iteration if paging was enabled.
  946. type Iter struct {
  947. err error
  948. pos int
  949. meta resultMetadata
  950. numRows int
  951. next *nextIter
  952. host *HostInfo
  953. framer *framer
  954. closed int32
  955. }
  956. // Host returns the host which the query was sent to.
  957. func (iter *Iter) Host() *HostInfo {
  958. return iter.host
  959. }
  960. // Columns returns the name and type of the selected columns.
  961. func (iter *Iter) Columns() []ColumnInfo {
  962. return iter.meta.columns
  963. }
  964. type Scanner interface {
  965. // Next advances the row pointer to point at the next row, the row is valid until
  966. // the next call of Next. It returns true if there is a row which is available to be
  967. // scanned into with Scan.
  968. // Next must be called before every call to Scan.
  969. Next() bool
  970. // Scan copies the current row's columns into dest. If the length of dest does not equal
  971. // the number of columns returned in the row an error is returned. If an error is encountered
  972. // when unmarshalling a column into the value in dest an error is returned and the row is invalidated
  973. // until the next call to Next.
  974. // Next must be called before calling Scan, if it is not an error is returned.
  975. Scan(...interface{}) error
  976. // Err returns the if there was one during iteration that resulted in iteration being unable to complete.
  977. // Err will also release resources held by the iterator, the Scanner should not used after being called.
  978. Err() error
  979. }
  980. type iterScanner struct {
  981. iter *Iter
  982. cols [][]byte
  983. valid bool
  984. }
  985. func (is *iterScanner) Next() bool {
  986. iter := is.iter
  987. if iter.err != nil {
  988. return false
  989. }
  990. if iter.pos >= iter.numRows {
  991. if iter.next != nil {
  992. is.iter = iter.next.fetch()
  993. return is.Next()
  994. }
  995. return false
  996. }
  997. for i := 0; i < len(is.cols); i++ {
  998. col, err := iter.readColumn()
  999. if err != nil {
  1000. iter.err = err
  1001. return false
  1002. }
  1003. is.cols[i] = col
  1004. }
  1005. iter.pos++
  1006. is.valid = true
  1007. return true
  1008. }
  1009. func scanColumn(p []byte, col ColumnInfo, dest []interface{}) (int, error) {
  1010. if dest[0] == nil {
  1011. return 1, nil
  1012. }
  1013. if col.TypeInfo.Type() == TypeTuple {
  1014. // this will panic, actually a bug, please report
  1015. tuple := col.TypeInfo.(TupleTypeInfo)
  1016. count := len(tuple.Elems)
  1017. // here we pass in a slice of the struct which has the number number of
  1018. // values as elements in the tuple
  1019. if err := Unmarshal(col.TypeInfo, p, dest[:count]); err != nil {
  1020. return 0, err
  1021. }
  1022. return count, nil
  1023. } else {
  1024. if err := Unmarshal(col.TypeInfo, p, dest[0]); err != nil {
  1025. return 0, err
  1026. }
  1027. return 1, nil
  1028. }
  1029. }
  1030. func (is *iterScanner) Scan(dest ...interface{}) error {
  1031. if !is.valid {
  1032. return errors.New("gocql: Scan called without calling Next")
  1033. }
  1034. iter := is.iter
  1035. // currently only support scanning into an expand tuple, such that its the same
  1036. // as scanning in more values from a single column
  1037. if len(dest) != iter.meta.actualColCount {
  1038. return fmt.Errorf("gocql: not enough columns to scan into: have %d want %d", len(dest), iter.meta.actualColCount)
  1039. }
  1040. // i is the current position in dest, could posible replace it and just use
  1041. // slices of dest
  1042. i := 0
  1043. var err error
  1044. for _, col := range iter.meta.columns {
  1045. var n int
  1046. n, err = scanColumn(is.cols[i], col, dest[i:])
  1047. if err != nil {
  1048. break
  1049. }
  1050. i += n
  1051. }
  1052. is.valid = false
  1053. return err
  1054. }
  1055. func (is *iterScanner) Err() error {
  1056. iter := is.iter
  1057. is.iter = nil
  1058. is.cols = nil
  1059. is.valid = false
  1060. return iter.Close()
  1061. }
  1062. // Scanner returns a row Scanner which provides an interface to scan rows in a manner which is
  1063. // similar to database/sql. The iter should NOT be used again after calling this method.
  1064. func (iter *Iter) Scanner() Scanner {
  1065. if iter == nil {
  1066. return nil
  1067. }
  1068. return &iterScanner{iter: iter, cols: make([][]byte, len(iter.meta.columns))}
  1069. }
  1070. func (iter *Iter) readColumn() ([]byte, error) {
  1071. return iter.framer.readBytesInternal()
  1072. }
  1073. // Scan consumes the next row of the iterator and copies the columns of the
  1074. // current row into the values pointed at by dest. Use nil as a dest value
  1075. // to skip the corresponding column. Scan might send additional queries
  1076. // to the database to retrieve the next set of rows if paging was enabled.
  1077. //
  1078. // Scan returns true if the row was successfully unmarshaled or false if the
  1079. // end of the result set was reached or if an error occurred. Close should
  1080. // be called afterwards to retrieve any potential errors.
  1081. func (iter *Iter) Scan(dest ...interface{}) bool {
  1082. if iter.err != nil {
  1083. return false
  1084. }
  1085. if iter.pos >= iter.numRows {
  1086. if iter.next != nil {
  1087. *iter = *iter.next.fetch()
  1088. return iter.Scan(dest...)
  1089. }
  1090. return false
  1091. }
  1092. if iter.next != nil && iter.pos == iter.next.pos {
  1093. go iter.next.fetch()
  1094. }
  1095. // currently only support scanning into an expand tuple, such that its the same
  1096. // as scanning in more values from a single column
  1097. if len(dest) != iter.meta.actualColCount {
  1098. iter.err = fmt.Errorf("gocql: not enough columns to scan into: have %d want %d", len(dest), iter.meta.actualColCount)
  1099. return false
  1100. }
  1101. // i is the current position in dest, could posible replace it and just use
  1102. // slices of dest
  1103. i := 0
  1104. for _, col := range iter.meta.columns {
  1105. colBytes, err := iter.readColumn()
  1106. if err != nil {
  1107. iter.err = err
  1108. return false
  1109. }
  1110. n, err := scanColumn(colBytes, col, dest[i:])
  1111. if err != nil {
  1112. iter.err = err
  1113. return false
  1114. }
  1115. i += n
  1116. }
  1117. iter.pos++
  1118. return true
  1119. }
  1120. // GetCustomPayload returns any parsed custom payload results if given in the
  1121. // response from Cassandra. Note that the result is not a copy.
  1122. //
  1123. // This additional feature of CQL Protocol v4
  1124. // allows additional results and query information to be returned by
  1125. // custom QueryHandlers running in your C* cluster.
  1126. // See https://datastax.github.io/java-driver/manual/custom_payloads/
  1127. func (iter *Iter) GetCustomPayload() map[string][]byte {
  1128. return iter.framer.header.customPayload
  1129. }
  1130. // Warnings returns any warnings generated if given in the response from Cassandra.
  1131. //
  1132. // This is only available starting with CQL Protocol v4.
  1133. func (iter *Iter) Warnings() []string {
  1134. if iter.framer != nil {
  1135. return iter.framer.header.warnings
  1136. }
  1137. return nil
  1138. }
  1139. // Close closes the iterator and returns any errors that happened during
  1140. // the query or the iteration.
  1141. func (iter *Iter) Close() error {
  1142. if atomic.CompareAndSwapInt32(&iter.closed, 0, 1) {
  1143. if iter.framer != nil {
  1144. iter.framer = nil
  1145. }
  1146. }
  1147. return iter.err
  1148. }
  1149. // WillSwitchPage detects if iterator reached end of current page
  1150. // and the next page is available.
  1151. func (iter *Iter) WillSwitchPage() bool {
  1152. return iter.pos >= iter.numRows && iter.next != nil
  1153. }
  1154. // checkErrAndNotFound handle error and NotFound in one method.
  1155. func (iter *Iter) checkErrAndNotFound() error {
  1156. if iter.err != nil {
  1157. return iter.err
  1158. } else if iter.numRows == 0 {
  1159. return ErrNotFound
  1160. }
  1161. return nil
  1162. }
  1163. // PageState return the current paging state for a query which can be used for
  1164. // subsequent quries to resume paging this point.
  1165. func (iter *Iter) PageState() []byte {
  1166. return iter.meta.pagingState
  1167. }
  1168. // NumRows returns the number of rows in this pagination, it will update when new
  1169. // pages are fetched, it is not the value of the total number of rows this iter
  1170. // will return unless there is only a single page returned.
  1171. func (iter *Iter) NumRows() int {
  1172. return iter.numRows
  1173. }
  1174. type nextIter struct {
  1175. qry Query
  1176. pos int
  1177. once sync.Once
  1178. next *Iter
  1179. conn *Conn
  1180. }
  1181. func (n *nextIter) fetch() *Iter {
  1182. n.once.Do(func() {
  1183. iter := n.qry.session.executor.attemptQuery(&n.qry, n.conn)
  1184. if iter != nil && iter.err == nil {
  1185. n.next = iter
  1186. } else {
  1187. n.next = n.qry.session.executeQuery(&n.qry)
  1188. }
  1189. })
  1190. return n.next
  1191. }
  1192. type Batch struct {
  1193. Type BatchType
  1194. Entries []BatchEntry
  1195. Cons Consistency
  1196. rt RetryPolicy
  1197. observer BatchObserver
  1198. attempts int
  1199. totalLatency int64
  1200. serialCons SerialConsistency
  1201. defaultTimestamp bool
  1202. defaultTimestampValue int64
  1203. context context.Context
  1204. keyspace string
  1205. }
  1206. // NewBatch creates a new batch operation without defaults from the cluster
  1207. //
  1208. // Depreicated: use session.NewBatch instead
  1209. func NewBatch(typ BatchType) *Batch {
  1210. return &Batch{Type: typ}
  1211. }
  1212. // NewBatch creates a new batch operation using defaults defined in the cluster
  1213. func (s *Session) NewBatch(typ BatchType) *Batch {
  1214. s.mu.RLock()
  1215. batch := &Batch{
  1216. Type: typ,
  1217. rt: s.cfg.RetryPolicy,
  1218. serialCons: s.cfg.SerialConsistency,
  1219. observer: s.batchObserver,
  1220. Cons: s.cons,
  1221. defaultTimestamp: s.cfg.DefaultTimestamp,
  1222. keyspace: s.cfg.Keyspace,
  1223. }
  1224. s.mu.RUnlock()
  1225. return batch
  1226. }
  1227. // Observer enables batch-level observer on this batch.
  1228. // The provided observer will be called every time this batched query is executed.
  1229. func (b *Batch) Observer(observer BatchObserver) *Batch {
  1230. b.observer = observer
  1231. return b
  1232. }
  1233. func (b *Batch) Keyspace() string {
  1234. return b.keyspace
  1235. }
  1236. // Attempts returns the number of attempts made to execute the batch.
  1237. func (b *Batch) Attempts() int {
  1238. return b.attempts
  1239. }
  1240. //Latency returns the average number of nanoseconds to execute a single attempt of the batch.
  1241. func (b *Batch) Latency() int64 {
  1242. if b.attempts > 0 {
  1243. return b.totalLatency / int64(b.attempts)
  1244. }
  1245. return 0
  1246. }
  1247. // GetConsistency returns the currently configured consistency level for the batch
  1248. // operation.
  1249. func (b *Batch) GetConsistency() Consistency {
  1250. return b.Cons
  1251. }
  1252. // SetConsistency sets the currently configured consistency level for the batch
  1253. // operation.
  1254. func (b *Batch) SetConsistency(c Consistency) {
  1255. b.Cons = c
  1256. }
  1257. // Query adds the query to the batch operation
  1258. func (b *Batch) Query(stmt string, args ...interface{}) {
  1259. b.Entries = append(b.Entries, BatchEntry{Stmt: stmt, Args: args})
  1260. }
  1261. // Bind adds the query to the batch operation and correlates it with a binding callback
  1262. // that will be invoked when the batch is executed. The binding callback allows the application
  1263. // to define which query argument values will be marshalled as part of the batch execution.
  1264. func (b *Batch) Bind(stmt string, bind func(q *QueryInfo) ([]interface{}, error)) {
  1265. b.Entries = append(b.Entries, BatchEntry{Stmt: stmt, binding: bind})
  1266. }
  1267. func (b *Batch) retryPolicy() RetryPolicy {
  1268. return b.rt
  1269. }
  1270. // RetryPolicy sets the retry policy to use when executing the batch operation
  1271. func (b *Batch) RetryPolicy(r RetryPolicy) *Batch {
  1272. b.rt = r
  1273. return b
  1274. }
  1275. // WithContext will set the context to use during a query, it will be used to
  1276. // timeout when waiting for responses from Cassandra.
  1277. func (b *Batch) WithContext(ctx context.Context) *Batch {
  1278. b.context = ctx
  1279. return b
  1280. }
  1281. // Context satisfies the ExecutableQuery interface.
  1282. func (b *Batch) Context() context.Context {
  1283. return b.context
  1284. }
  1285. // Size returns the number of batch statements to be executed by the batch operation.
  1286. func (b *Batch) Size() int {
  1287. return len(b.Entries)
  1288. }
  1289. // SerialConsistency sets the consistency level for the
  1290. // serial phase of conditional updates. That consistency can only be
  1291. // either SERIAL or LOCAL_SERIAL and if not present, it defaults to
  1292. // SERIAL. This option will be ignored for anything else that a
  1293. // conditional update/insert.
  1294. //
  1295. // Only available for protocol 3 and above
  1296. func (b *Batch) SerialConsistency(cons SerialConsistency) *Batch {
  1297. b.serialCons = cons
  1298. return b
  1299. }
  1300. // DefaultTimestamp will enable the with default timestamp flag on the query.
  1301. // If enable, this will replace the server side assigned
  1302. // timestamp as default timestamp. Note that a timestamp in the query itself
  1303. // will still override this timestamp. This is entirely optional.
  1304. //
  1305. // Only available on protocol >= 3
  1306. func (b *Batch) DefaultTimestamp(enable bool) *Batch {
  1307. b.defaultTimestamp = enable
  1308. return b
  1309. }
  1310. // WithTimestamp will enable the with default timestamp flag on the query
  1311. // like DefaultTimestamp does. But also allows to define value for timestamp.
  1312. // It works the same way as USING TIMESTAMP in the query itself, but
  1313. // should not break prepared query optimization
  1314. //
  1315. // Only available on protocol >= 3
  1316. func (b *Batch) WithTimestamp(timestamp int64) *Batch {
  1317. b.DefaultTimestamp(true)
  1318. b.defaultTimestampValue = timestamp
  1319. return b
  1320. }
  1321. func (b *Batch) attempt(keyspace string, end, start time.Time, iter *Iter, host *HostInfo) {
  1322. b.attempts++
  1323. b.totalLatency += end.Sub(start).Nanoseconds()
  1324. // TODO: track latencies per host and things as well instead of just total
  1325. if b.observer == nil {
  1326. return
  1327. }
  1328. statements := make([]string, len(b.Entries))
  1329. for i, entry := range b.Entries {
  1330. statements[i] = entry.Stmt
  1331. }
  1332. b.observer.ObserveBatch(b.context, ObservedBatch{
  1333. Keyspace: keyspace,
  1334. Statements: statements,
  1335. Start: start,
  1336. End: end,
  1337. // Rows not used in batch observations // TODO - might be able to support it when using BatchCAS
  1338. Host: host,
  1339. Err: iter.err,
  1340. Attempt: b.attempts,
  1341. })
  1342. }
  1343. func (b *Batch) GetRoutingKey() ([]byte, error) {
  1344. // TODO: use the first statement in the batch as the routing key?
  1345. return nil, nil
  1346. }
  1347. type BatchType byte
  1348. const (
  1349. LoggedBatch BatchType = 0
  1350. UnloggedBatch BatchType = 1
  1351. CounterBatch BatchType = 2
  1352. )
  1353. type BatchEntry struct {
  1354. Stmt string
  1355. Args []interface{}
  1356. binding func(q *QueryInfo) ([]interface{}, error)
  1357. }
  1358. type ColumnInfo struct {
  1359. Keyspace string
  1360. Table string
  1361. Name string
  1362. TypeInfo TypeInfo
  1363. }
  1364. func (c ColumnInfo) String() string {
  1365. return fmt.Sprintf("[column keyspace=%s table=%s name=%s type=%v]", c.Keyspace, c.Table, c.Name, c.TypeInfo)
  1366. }
  1367. // routing key indexes LRU cache
  1368. type routingKeyInfoLRU struct {
  1369. lru *lru.Cache
  1370. mu sync.Mutex
  1371. }
  1372. type routingKeyInfo struct {
  1373. indexes []int
  1374. types []TypeInfo
  1375. }
  1376. func (r *routingKeyInfo) String() string {
  1377. return fmt.Sprintf("routing key index=%v types=%v", r.indexes, r.types)
  1378. }
  1379. func (r *routingKeyInfoLRU) Remove(key string) {
  1380. r.mu.Lock()
  1381. r.lru.Remove(key)
  1382. r.mu.Unlock()
  1383. }
  1384. //Max adjusts the maximum size of the cache and cleans up the oldest records if
  1385. //the new max is lower than the previous value. Not concurrency safe.
  1386. func (r *routingKeyInfoLRU) Max(max int) {
  1387. r.mu.Lock()
  1388. for r.lru.Len() > max {
  1389. r.lru.RemoveOldest()
  1390. }
  1391. r.lru.MaxEntries = max
  1392. r.mu.Unlock()
  1393. }
  1394. type inflightCachedEntry struct {
  1395. wg sync.WaitGroup
  1396. err error
  1397. value interface{}
  1398. }
  1399. // Tracer is the interface implemented by query tracers. Tracers have the
  1400. // ability to obtain a detailed event log of all events that happened during
  1401. // the execution of a query from Cassandra. Gathering this information might
  1402. // be essential for debugging and optimizing queries, but this feature should
  1403. // not be used on production systems with very high load.
  1404. type Tracer interface {
  1405. Trace(traceId []byte)
  1406. }
  1407. type traceWriter struct {
  1408. session *Session
  1409. w io.Writer
  1410. mu sync.Mutex
  1411. }
  1412. // NewTraceWriter returns a simple Tracer implementation that outputs
  1413. // the event log in a textual format.
  1414. func NewTraceWriter(session *Session, w io.Writer) Tracer {
  1415. return &traceWriter{session: session, w: w}
  1416. }
  1417. func (t *traceWriter) Trace(traceId []byte) {
  1418. var (
  1419. coordinator string
  1420. duration int
  1421. )
  1422. iter := t.session.control.query(`SELECT coordinator, duration
  1423. FROM system_traces.sessions
  1424. WHERE session_id = ?`, traceId)
  1425. iter.Scan(&coordinator, &duration)
  1426. if err := iter.Close(); err != nil {
  1427. t.mu.Lock()
  1428. fmt.Fprintln(t.w, "Error:", err)
  1429. t.mu.Unlock()
  1430. return
  1431. }
  1432. var (
  1433. timestamp time.Time
  1434. activity string
  1435. source string
  1436. elapsed int
  1437. )
  1438. t.mu.Lock()
  1439. defer t.mu.Unlock()
  1440. fmt.Fprintf(t.w, "Tracing session %016x (coordinator: %s, duration: %v):\n",
  1441. traceId, coordinator, time.Duration(duration)*time.Microsecond)
  1442. iter = t.session.control.query(`SELECT event_id, activity, source, source_elapsed
  1443. FROM system_traces.events
  1444. WHERE session_id = ?`, traceId)
  1445. for iter.Scan(&timestamp, &activity, &source, &elapsed) {
  1446. fmt.Fprintf(t.w, "%s: %s (source: %s, elapsed: %d)\n",
  1447. timestamp.Format("2006/01/02 15:04:05.999999"), activity, source, elapsed)
  1448. }
  1449. if err := iter.Close(); err != nil {
  1450. fmt.Fprintln(t.w, "Error:", err)
  1451. }
  1452. }
  1453. type ObservedQuery struct {
  1454. Keyspace string
  1455. Statement string
  1456. Start time.Time // time immediately before the query was called
  1457. End time.Time // time immediately after the query returned
  1458. // Rows is the number of rows in the current iter.
  1459. // In paginated queries, rows from previous scans are not counted.
  1460. // Rows is not used in batch queries and remains at the default value
  1461. Rows int
  1462. // Host is the informations about the host that performed the query
  1463. Host *HostInfo
  1464. // Err is the error in the query.
  1465. // It only tracks network errors or errors of bad cassandra syntax, in particular selects with no match return nil error
  1466. Err error
  1467. // Attempt contains the number of times the query has been attempted so far.
  1468. Attempt int
  1469. }
  1470. // QueryObserver is the interface implemented by query observers / stat collectors.
  1471. //
  1472. // Experimental, this interface and use may change
  1473. type QueryObserver interface {
  1474. // ObserveQuery gets called on every query to cassandra, including all queries in an iterator when paging is enabled.
  1475. // It doesn't get called if there is no query because the session is closed or there are no connections available.
  1476. // The error reported only shows query errors, i.e. if a SELECT is valid but finds no matches it will be nil.
  1477. ObserveQuery(context.Context, ObservedQuery)
  1478. }
  1479. type ObservedBatch struct {
  1480. Keyspace string
  1481. Statements []string
  1482. Start time.Time // time immediately before the batch query was called
  1483. End time.Time // time immediately after the batch query returned
  1484. // Host is the informations about the host that performed the batch
  1485. Host *HostInfo
  1486. // Err is the error in the batch query.
  1487. // It only tracks network errors or errors of bad cassandra syntax, in particular selects with no match return nil error
  1488. Err error
  1489. // Attempt contains the number of times the query has been attempted so far.
  1490. Attempt int
  1491. }
  1492. // BatchObserver is the interface implemented by batch observers / stat collectors.
  1493. type BatchObserver interface {
  1494. // ObserveBatch gets called on every batch query to cassandra.
  1495. // It also gets called once for each query in a batch.
  1496. // It doesn't get called if there is no query because the session is closed or there are no connections available.
  1497. // The error reported only shows query errors, i.e. if a SELECT is valid but finds no matches it will be nil.
  1498. // Unlike QueryObserver.ObserveQuery it does no reporting on rows read.
  1499. ObserveBatch(context.Context, ObservedBatch)
  1500. }
  1501. type ObservedConnect struct {
  1502. // Host is the information about the host about to connect
  1503. Host *HostInfo
  1504. Start time.Time // time immediately before the dial is called
  1505. End time.Time // time immediately after the dial returned
  1506. // Err is the connection error (if any)
  1507. Err error
  1508. }
  1509. // ConnectObserver is the interface implemented by connect observers / stat collectors.
  1510. type ConnectObserver interface {
  1511. // ObserveConnect gets called when a new connection to cassandra is made.
  1512. ObserveConnect(ObservedConnect)
  1513. }
  1514. type Error struct {
  1515. Code int
  1516. Message string
  1517. }
  1518. func (e Error) Error() string {
  1519. return e.Message
  1520. }
  1521. var (
  1522. ErrNotFound = errors.New("not found")
  1523. ErrUnavailable = errors.New("unavailable")
  1524. ErrUnsupported = errors.New("feature not supported")
  1525. ErrTooManyStmts = errors.New("too many statements")
  1526. ErrUseStmt = errors.New("use statements aren't supported. Please see https://github.com/gocql/gocql for explanation.")
  1527. ErrSessionClosed = errors.New("session has been closed")
  1528. ErrNoConnections = errors.New("gocql: no hosts available in the pool")
  1529. ErrNoKeyspace = errors.New("no keyspace provided")
  1530. ErrKeyspaceDoesNotExist = errors.New("keyspace does not exist")
  1531. ErrNoMetadata = errors.New("no metadata available")
  1532. )
  1533. type ErrProtocol struct{ error }
  1534. func NewErrProtocol(format string, args ...interface{}) error {
  1535. return ErrProtocol{fmt.Errorf(format, args...)}
  1536. }
  1537. // BatchSizeMaximum is the maximum number of statements a batch operation can have.
  1538. // This limit is set by cassandra and could change in the future.
  1539. const BatchSizeMaximum = 65535