cassandra_test.go 83 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366136713681369137013711372137313741375137613771378137913801381138213831384138513861387138813891390139113921393139413951396139713981399140014011402140314041405140614071408140914101411141214131414141514161417141814191420142114221423142414251426142714281429143014311432143314341435143614371438143914401441144214431444144514461447144814491450145114521453145414551456145714581459146014611462146314641465146614671468146914701471147214731474147514761477147814791480148114821483148414851486148714881489149014911492149314941495149614971498149915001501150215031504150515061507150815091510151115121513151415151516151715181519152015211522152315241525152615271528152915301531153215331534153515361537153815391540154115421543154415451546154715481549155015511552155315541555155615571558155915601561156215631564156515661567156815691570157115721573157415751576157715781579158015811582158315841585158615871588158915901591159215931594159515961597159815991600160116021603160416051606160716081609161016111612161316141615161616171618161916201621162216231624162516261627162816291630163116321633163416351636163716381639164016411642164316441645164616471648164916501651165216531654165516561657165816591660166116621663166416651666166716681669167016711672167316741675167616771678167916801681168216831684168516861687168816891690169116921693169416951696169716981699170017011702170317041705170617071708170917101711171217131714171517161717171817191720172117221723172417251726172717281729173017311732173317341735173617371738173917401741174217431744174517461747174817491750175117521753175417551756175717581759176017611762176317641765176617671768176917701771177217731774177517761777177817791780178117821783178417851786178717881789179017911792179317941795179617971798179918001801180218031804180518061807180818091810181118121813181418151816181718181819182018211822182318241825182618271828182918301831183218331834183518361837183818391840184118421843184418451846184718481849185018511852185318541855185618571858185918601861186218631864186518661867186818691870187118721873187418751876187718781879188018811882188318841885188618871888188918901891189218931894189518961897189818991900190119021903190419051906190719081909191019111912191319141915191619171918191919201921192219231924192519261927192819291930193119321933193419351936193719381939194019411942194319441945194619471948194919501951195219531954195519561957195819591960196119621963196419651966196719681969197019711972197319741975197619771978197919801981198219831984198519861987198819891990199119921993199419951996199719981999200020012002200320042005200620072008200920102011201220132014201520162017201820192020202120222023202420252026202720282029203020312032203320342035203620372038203920402041204220432044204520462047204820492050205120522053205420552056205720582059206020612062206320642065206620672068206920702071207220732074207520762077207820792080208120822083208420852086208720882089209020912092209320942095209620972098209921002101210221032104210521062107210821092110211121122113211421152116211721182119212021212122212321242125212621272128212921302131213221332134213521362137213821392140214121422143214421452146214721482149215021512152215321542155215621572158215921602161216221632164216521662167216821692170217121722173217421752176217721782179218021812182218321842185218621872188218921902191219221932194219521962197219821992200220122022203220422052206220722082209221022112212221322142215221622172218221922202221222222232224222522262227222822292230223122322233223422352236223722382239224022412242224322442245224622472248224922502251225222532254225522562257225822592260226122622263226422652266226722682269227022712272227322742275227622772278227922802281228222832284228522862287228822892290229122922293229422952296229722982299230023012302230323042305230623072308230923102311231223132314231523162317231823192320232123222323232423252326232723282329233023312332233323342335233623372338233923402341234223432344234523462347234823492350235123522353235423552356235723582359236023612362236323642365236623672368236923702371237223732374237523762377237823792380238123822383238423852386238723882389239023912392239323942395239623972398239924002401240224032404240524062407240824092410241124122413241424152416241724182419242024212422242324242425242624272428242924302431243224332434243524362437243824392440244124422443244424452446244724482449245024512452245324542455245624572458245924602461246224632464246524662467246824692470247124722473247424752476247724782479248024812482248324842485248624872488248924902491249224932494249524962497249824992500250125022503250425052506250725082509251025112512251325142515251625172518251925202521252225232524252525262527252825292530253125322533253425352536253725382539254025412542254325442545254625472548254925502551255225532554255525562557255825592560256125622563256425652566256725682569257025712572257325742575257625772578257925802581258225832584258525862587258825892590259125922593259425952596259725982599260026012602260326042605260626072608260926102611261226132614261526162617261826192620262126222623262426252626262726282629263026312632263326342635263626372638263926402641264226432644264526462647264826492650265126522653265426552656265726582659
  1. // +build all integration
  2. package gocql
  3. import (
  4. "bytes"
  5. "context"
  6. "io"
  7. "math"
  8. "math/big"
  9. "net"
  10. "reflect"
  11. "strconv"
  12. "strings"
  13. "sync"
  14. "testing"
  15. "time"
  16. "unicode"
  17. "gopkg.in/inf.v0"
  18. )
  19. // TestAuthentication verifies that gocql will work with a host configured to only accept authenticated connections
  20. func TestAuthentication(t *testing.T) {
  21. if *flagProto < 2 {
  22. t.Skip("Authentication is not supported with protocol < 2")
  23. }
  24. if !*flagRunAuthTest {
  25. t.Skip("Authentication is not configured in the target cluster")
  26. }
  27. cluster := createCluster()
  28. cluster.Authenticator = PasswordAuthenticator{
  29. Username: "cassandra",
  30. Password: "cassandra",
  31. }
  32. session, err := cluster.CreateSession()
  33. if err != nil {
  34. t.Fatalf("Authentication error: %s", err)
  35. }
  36. session.Close()
  37. }
  38. //TestRingDiscovery makes sure that you can autodiscover other cluster members when you seed a cluster config with just one node
  39. func TestRingDiscovery(t *testing.T) {
  40. cluster := createCluster()
  41. cluster.Hosts = clusterHosts[:1]
  42. session := createSessionFromCluster(cluster, t)
  43. defer session.Close()
  44. if *clusterSize > 1 {
  45. // wait for autodiscovery to update the pool with the list of known hosts
  46. time.Sleep(*flagAutoWait)
  47. }
  48. session.pool.mu.RLock()
  49. defer session.pool.mu.RUnlock()
  50. size := len(session.pool.hostConnPools)
  51. if *clusterSize != size {
  52. for p, pool := range session.pool.hostConnPools {
  53. t.Logf("p=%q host=%v ips=%s", p, pool.host, pool.host.Peer().String())
  54. }
  55. t.Errorf("Expected a cluster size of %d, but actual size was %d", *clusterSize, size)
  56. }
  57. }
  58. func TestEmptyHosts(t *testing.T) {
  59. cluster := createCluster()
  60. cluster.Hosts = nil
  61. if session, err := cluster.CreateSession(); err == nil {
  62. session.Close()
  63. t.Error("expected err, got nil")
  64. }
  65. }
  66. func TestInvalidPeerEntry(t *testing.T) {
  67. session := createSession(t)
  68. // rack, release_version, schema_version, tokens are all null
  69. query := session.Query("INSERT into system.peers (peer, data_center, host_id, rpc_address) VALUES (?, ?, ?, ?)",
  70. "169.254.235.45",
  71. "datacenter1",
  72. "35c0ec48-5109-40fd-9281-9e9d4add2f1e",
  73. "169.254.235.45",
  74. )
  75. if err := query.Exec(); err != nil {
  76. t.Fatal(err)
  77. }
  78. session.Close()
  79. cluster := createCluster()
  80. cluster.PoolConfig.HostSelectionPolicy = TokenAwareHostPolicy(RoundRobinHostPolicy())
  81. session = createSessionFromCluster(cluster, t)
  82. defer func() {
  83. session.Query("DELETE from system.peers where peer = ?", "169.254.235.45").Exec()
  84. session.Close()
  85. }()
  86. // check we can perform a query
  87. iter := session.Query("select peer from system.peers").Iter()
  88. var peer string
  89. for iter.Scan(&peer) {
  90. }
  91. if err := iter.Close(); err != nil {
  92. t.Fatal(err)
  93. }
  94. }
  95. //TestUseStatementError checks to make sure the correct error is returned when the user tries to execute a use statement.
  96. func TestUseStatementError(t *testing.T) {
  97. session := createSession(t)
  98. defer session.Close()
  99. if err := session.Query("USE gocql_test").Exec(); err != nil {
  100. if err != ErrUseStmt {
  101. t.Fatalf("expected ErrUseStmt, got " + err.Error())
  102. }
  103. } else {
  104. t.Fatal("expected err, got nil.")
  105. }
  106. }
  107. //TestInvalidKeyspace checks that an invalid keyspace will return promptly and without a flood of connections
  108. func TestInvalidKeyspace(t *testing.T) {
  109. cluster := createCluster()
  110. cluster.Keyspace = "invalidKeyspace"
  111. session, err := cluster.CreateSession()
  112. if err != nil {
  113. if err != ErrNoConnectionsStarted {
  114. t.Fatalf("Expected ErrNoConnections but got %v", err)
  115. }
  116. } else {
  117. session.Close() //Clean up the session
  118. t.Fatal("expected err, got nil.")
  119. }
  120. }
  121. func TestTracing(t *testing.T) {
  122. session := createSession(t)
  123. defer session.Close()
  124. if err := createTable(session, `CREATE TABLE gocql_test.trace (id int primary key)`); err != nil {
  125. t.Fatal("create:", err)
  126. }
  127. buf := &bytes.Buffer{}
  128. trace := NewTraceWriter(session, buf)
  129. if err := session.Query(`INSERT INTO trace (id) VALUES (?)`, 42).Trace(trace).Exec(); err != nil {
  130. t.Fatal("insert:", err)
  131. } else if buf.Len() == 0 {
  132. t.Fatal("insert: failed to obtain any tracing")
  133. }
  134. buf.Reset()
  135. var value int
  136. if err := session.Query(`SELECT id FROM trace WHERE id = ?`, 42).Trace(trace).Scan(&value); err != nil {
  137. t.Fatal("select:", err)
  138. } else if value != 42 {
  139. t.Fatalf("value: expected %d, got %d", 42, value)
  140. } else if buf.Len() == 0 {
  141. t.Fatal("select: failed to obtain any tracing")
  142. }
  143. // also works from session tracer
  144. session.SetTrace(trace)
  145. buf.Reset()
  146. if err := session.Query(`SELECT id FROM trace WHERE id = ?`, 42).Scan(&value); err != nil {
  147. t.Fatal("select:", err)
  148. }
  149. if buf.Len() == 0 {
  150. t.Fatal("select: failed to obtain any tracing")
  151. }
  152. }
  153. func TestPaging(t *testing.T) {
  154. if *flagProto == 1 {
  155. t.Skip("Paging not supported. Please use Cassandra >= 2.0")
  156. }
  157. session := createSession(t)
  158. defer session.Close()
  159. if err := createTable(session, "CREATE TABLE gocql_test.paging (id int primary key)"); err != nil {
  160. t.Fatal("create table:", err)
  161. }
  162. for i := 0; i < 100; i++ {
  163. if err := session.Query("INSERT INTO paging (id) VALUES (?)", i).Exec(); err != nil {
  164. t.Fatal("insert:", err)
  165. }
  166. }
  167. iter := session.Query("SELECT id FROM paging").PageSize(10).Iter()
  168. var id int
  169. count := 0
  170. for iter.Scan(&id) {
  171. count++
  172. }
  173. if err := iter.Close(); err != nil {
  174. t.Fatal("close:", err)
  175. }
  176. if count != 100 {
  177. t.Fatalf("expected %d, got %d", 100, count)
  178. }
  179. }
  180. func TestCAS(t *testing.T) {
  181. if *flagProto == 1 {
  182. t.Skip("lightweight transactions not supported. Please use Cassandra >= 2.0")
  183. }
  184. cluster := createCluster()
  185. cluster.SerialConsistency = LocalSerial
  186. session := createSessionFromCluster(cluster, t)
  187. defer session.Close()
  188. if err := createTable(session, `CREATE TABLE gocql_test.cas_table (
  189. title varchar,
  190. revid timeuuid,
  191. last_modified timestamp,
  192. PRIMARY KEY (title, revid)
  193. )`); err != nil {
  194. t.Fatal("create:", err)
  195. }
  196. title, revid, modified := "baz", TimeUUID(), time.Now()
  197. var titleCAS string
  198. var revidCAS UUID
  199. var modifiedCAS time.Time
  200. if applied, err := session.Query(`INSERT INTO cas_table (title, revid, last_modified)
  201. VALUES (?, ?, ?) IF NOT EXISTS`,
  202. title, revid, modified).ScanCAS(&titleCAS, &revidCAS, &modifiedCAS); err != nil {
  203. t.Fatal("insert:", err)
  204. } else if !applied {
  205. t.Fatal("insert should have been applied")
  206. }
  207. if applied, err := session.Query(`INSERT INTO cas_table (title, revid, last_modified)
  208. VALUES (?, ?, ?) IF NOT EXISTS`,
  209. title, revid, modified).ScanCAS(&titleCAS, &revidCAS, &modifiedCAS); err != nil {
  210. t.Fatal("insert:", err)
  211. } else if applied {
  212. t.Fatal("insert should not have been applied")
  213. } else if title != titleCAS || revid != revidCAS {
  214. t.Fatalf("expected %s/%v/%v but got %s/%v/%v", title, revid, modified, titleCAS, revidCAS, modifiedCAS)
  215. }
  216. tenSecondsLater := modified.Add(10 * time.Second)
  217. if applied, err := session.Query(`DELETE FROM cas_table WHERE title = ? and revid = ? IF last_modified = ?`,
  218. title, revid, tenSecondsLater).ScanCAS(&modifiedCAS); err != nil {
  219. t.Fatal("delete:", err)
  220. } else if applied {
  221. t.Fatal("delete should have not been applied")
  222. }
  223. if modifiedCAS.Unix() != tenSecondsLater.Add(-10*time.Second).Unix() {
  224. t.Fatalf("Was expecting modified CAS to be %v; but was one second later", modifiedCAS.UTC())
  225. }
  226. if _, err := session.Query(`DELETE FROM cas_table WHERE title = ? and revid = ? IF last_modified = ?`,
  227. title, revid, tenSecondsLater).ScanCAS(); !strings.HasPrefix(err.Error(), "gocql: not enough columns to scan into") {
  228. t.Fatalf("delete: was expecting count mismatch error but got: %q", err.Error())
  229. }
  230. if applied, err := session.Query(`DELETE FROM cas_table WHERE title = ? and revid = ? IF last_modified = ?`,
  231. title, revid, modified).ScanCAS(&modifiedCAS); err != nil {
  232. t.Fatal("delete:", err)
  233. } else if !applied {
  234. t.Fatal("delete should have been applied")
  235. }
  236. if err := session.Query(`TRUNCATE cas_table`).Exec(); err != nil {
  237. t.Fatal("truncate:", err)
  238. }
  239. successBatch := session.NewBatch(LoggedBatch)
  240. successBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES (?, ?, ?) IF NOT EXISTS", title, revid, modified)
  241. if applied, _, err := session.ExecuteBatchCAS(successBatch, &titleCAS, &revidCAS, &modifiedCAS); err != nil {
  242. t.Fatal("insert:", err)
  243. } else if !applied {
  244. t.Fatalf("insert should have been applied: title=%v revID=%v modified=%v", titleCAS, revidCAS, modifiedCAS)
  245. }
  246. successBatch = session.NewBatch(LoggedBatch)
  247. successBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES (?, ?, ?) IF NOT EXISTS", title+"_foo", revid, modified)
  248. casMap := make(map[string]interface{})
  249. if applied, _, err := session.MapExecuteBatchCAS(successBatch, casMap); err != nil {
  250. t.Fatal("insert:", err)
  251. } else if !applied {
  252. t.Fatal("insert should have been applied")
  253. }
  254. failBatch := session.NewBatch(LoggedBatch)
  255. failBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES (?, ?, ?) IF NOT EXISTS", title, revid, modified)
  256. if applied, _, err := session.ExecuteBatchCAS(successBatch, &titleCAS, &revidCAS, &modifiedCAS); err != nil {
  257. t.Fatal("insert:", err)
  258. } else if applied {
  259. t.Fatalf("insert should have been applied: title=%v revID=%v modified=%v", titleCAS, revidCAS, modifiedCAS)
  260. }
  261. insertBatch := session.NewBatch(LoggedBatch)
  262. insertBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES ('_foo', 2c3af400-73a4-11e5-9381-29463d90c3f0, DATEOF(NOW()))")
  263. insertBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES ('_foo', 3e4ad2f1-73a4-11e5-9381-29463d90c3f0, DATEOF(NOW()))")
  264. if err := session.ExecuteBatch(insertBatch); err != nil {
  265. t.Fatal("insert:", err)
  266. }
  267. failBatch = session.NewBatch(LoggedBatch)
  268. failBatch.Query("UPDATE cas_table SET last_modified = DATEOF(NOW()) WHERE title='_foo' AND revid=2c3af400-73a4-11e5-9381-29463d90c3f0 IF last_modified=DATEOF(NOW());")
  269. failBatch.Query("UPDATE cas_table SET last_modified = DATEOF(NOW()) WHERE title='_foo' AND revid=3e4ad2f1-73a4-11e5-9381-29463d90c3f0 IF last_modified=DATEOF(NOW());")
  270. if applied, iter, err := session.ExecuteBatchCAS(failBatch, &titleCAS, &revidCAS, &modifiedCAS); err != nil {
  271. t.Fatal("insert:", err)
  272. } else if applied {
  273. t.Fatalf("insert should have been applied: title=%v revID=%v modified=%v", titleCAS, revidCAS, modifiedCAS)
  274. } else {
  275. if scan := iter.Scan(&applied, &titleCAS, &revidCAS, &modifiedCAS); scan && applied {
  276. t.Fatalf("insert should have been applied: title=%v revID=%v modified=%v", titleCAS, revidCAS, modifiedCAS)
  277. } else if !scan {
  278. t.Fatal("should have scanned another row")
  279. }
  280. if err := iter.Close(); err != nil {
  281. t.Fatal("scan:", err)
  282. }
  283. }
  284. }
  285. func TestMapScanCAS(t *testing.T) {
  286. if *flagProto == 1 {
  287. t.Skip("lightweight transactions not supported. Please use Cassandra >= 2.0")
  288. }
  289. session := createSession(t)
  290. defer session.Close()
  291. if err := createTable(session, `CREATE TABLE gocql_test.cas_table2 (
  292. title varchar,
  293. revid timeuuid,
  294. last_modified timestamp,
  295. deleted boolean,
  296. PRIMARY KEY (title, revid)
  297. )`); err != nil {
  298. t.Fatal("create:", err)
  299. }
  300. title, revid, modified, deleted := "baz", TimeUUID(), time.Now(), false
  301. mapCAS := map[string]interface{}{}
  302. if applied, err := session.Query(`INSERT INTO cas_table2 (title, revid, last_modified, deleted)
  303. VALUES (?, ?, ?, ?) IF NOT EXISTS`,
  304. title, revid, modified, deleted).MapScanCAS(mapCAS); err != nil {
  305. t.Fatal("insert:", err)
  306. } else if !applied {
  307. t.Fatalf("insert should have been applied: title=%v revID=%v modified=%v", title, revid, modified)
  308. }
  309. mapCAS = map[string]interface{}{}
  310. if applied, err := session.Query(`INSERT INTO cas_table2 (title, revid, last_modified, deleted)
  311. VALUES (?, ?, ?, ?) IF NOT EXISTS`,
  312. title, revid, modified, deleted).MapScanCAS(mapCAS); err != nil {
  313. t.Fatal("insert:", err)
  314. } else if applied {
  315. t.Fatalf("insert should have been applied: title=%v revID=%v modified=%v", title, revid, modified)
  316. } else if title != mapCAS["title"] || revid != mapCAS["revid"] || deleted != mapCAS["deleted"] {
  317. t.Fatalf("expected %s/%v/%v/%v but got %s/%v/%v%v", title, revid, modified, false, mapCAS["title"], mapCAS["revid"], mapCAS["last_modified"], mapCAS["deleted"])
  318. }
  319. }
  320. func TestBatch(t *testing.T) {
  321. if *flagProto == 1 {
  322. t.Skip("atomic batches not supported. Please use Cassandra >= 2.0")
  323. }
  324. session := createSession(t)
  325. defer session.Close()
  326. if err := createTable(session, `CREATE TABLE gocql_test.batch_table (id int primary key)`); err != nil {
  327. t.Fatal("create table:", err)
  328. }
  329. batch := NewBatch(LoggedBatch)
  330. for i := 0; i < 100; i++ {
  331. batch.Query(`INSERT INTO batch_table (id) VALUES (?)`, i)
  332. }
  333. if err := session.ExecuteBatch(batch); err != nil {
  334. t.Fatal("execute batch:", err)
  335. }
  336. count := 0
  337. if err := session.Query(`SELECT COUNT(*) FROM batch_table`).Scan(&count); err != nil {
  338. t.Fatal("select count:", err)
  339. } else if count != 100 {
  340. t.Fatalf("count: expected %d, got %d\n", 100, count)
  341. }
  342. }
  343. func TestUnpreparedBatch(t *testing.T) {
  344. t.Skip("FLAKE skipping")
  345. if *flagProto == 1 {
  346. t.Skip("atomic batches not supported. Please use Cassandra >= 2.0")
  347. }
  348. session := createSession(t)
  349. defer session.Close()
  350. if err := createTable(session, `CREATE TABLE gocql_test.batch_unprepared (id int primary key, c counter)`); err != nil {
  351. t.Fatal("create table:", err)
  352. }
  353. var batch *Batch
  354. if *flagProto == 2 {
  355. batch = NewBatch(CounterBatch)
  356. } else {
  357. batch = NewBatch(UnloggedBatch)
  358. }
  359. for i := 0; i < 100; i++ {
  360. batch.Query(`UPDATE batch_unprepared SET c = c + 1 WHERE id = 1`)
  361. }
  362. if err := session.ExecuteBatch(batch); err != nil {
  363. t.Fatal("execute batch:", err)
  364. }
  365. count := 0
  366. if err := session.Query(`SELECT COUNT(*) FROM batch_unprepared`).Scan(&count); err != nil {
  367. t.Fatal("select count:", err)
  368. } else if count != 1 {
  369. t.Fatalf("count: expected %d, got %d\n", 100, count)
  370. }
  371. if err := session.Query(`SELECT c FROM batch_unprepared`).Scan(&count); err != nil {
  372. t.Fatal("select count:", err)
  373. } else if count != 100 {
  374. t.Fatalf("count: expected %d, got %d\n", 100, count)
  375. }
  376. }
  377. // TestBatchLimit tests gocql to make sure batch operations larger than the maximum
  378. // statement limit are not submitted to a cassandra node.
  379. func TestBatchLimit(t *testing.T) {
  380. if *flagProto == 1 {
  381. t.Skip("atomic batches not supported. Please use Cassandra >= 2.0")
  382. }
  383. session := createSession(t)
  384. defer session.Close()
  385. if err := createTable(session, `CREATE TABLE gocql_test.batch_table2 (id int primary key)`); err != nil {
  386. t.Fatal("create table:", err)
  387. }
  388. batch := NewBatch(LoggedBatch)
  389. for i := 0; i < 65537; i++ {
  390. batch.Query(`INSERT INTO batch_table2 (id) VALUES (?)`, i)
  391. }
  392. if err := session.ExecuteBatch(batch); err != ErrTooManyStmts {
  393. t.Fatal("gocql attempted to execute a batch larger than the support limit of statements.")
  394. }
  395. }
  396. func TestWhereIn(t *testing.T) {
  397. session := createSession(t)
  398. defer session.Close()
  399. if err := createTable(session, `CREATE TABLE gocql_test.where_in_table (id int, cluster int, primary key (id,cluster))`); err != nil {
  400. t.Fatal("create table:", err)
  401. }
  402. if err := session.Query("INSERT INTO where_in_table (id, cluster) VALUES (?,?)", 100, 200).Exec(); err != nil {
  403. t.Fatal("insert:", err)
  404. }
  405. iter := session.Query("SELECT * FROM where_in_table WHERE id = ? AND cluster IN (?)", 100, 200).Iter()
  406. var id, cluster int
  407. count := 0
  408. for iter.Scan(&id, &cluster) {
  409. count++
  410. }
  411. if id != 100 || cluster != 200 {
  412. t.Fatalf("Was expecting id and cluster to be (100,200) but were (%d,%d)", id, cluster)
  413. }
  414. }
  415. // TestTooManyQueryArgs tests to make sure the library correctly handles the application level bug
  416. // whereby too many query arguments are passed to a query
  417. func TestTooManyQueryArgs(t *testing.T) {
  418. if *flagProto == 1 {
  419. t.Skip("atomic batches not supported. Please use Cassandra >= 2.0")
  420. }
  421. session := createSession(t)
  422. defer session.Close()
  423. if err := createTable(session, `CREATE TABLE gocql_test.too_many_query_args (id int primary key, value int)`); err != nil {
  424. t.Fatal("create table:", err)
  425. }
  426. _, err := session.Query(`SELECT * FROM too_many_query_args WHERE id = ?`, 1, 2).Iter().SliceMap()
  427. if err == nil {
  428. t.Fatal("'`SELECT * FROM too_many_query_args WHERE id = ?`, 1, 2' should return an error")
  429. }
  430. batch := session.NewBatch(UnloggedBatch)
  431. batch.Query("INSERT INTO too_many_query_args (id, value) VALUES (?, ?)", 1, 2, 3)
  432. err = session.ExecuteBatch(batch)
  433. if err == nil {
  434. t.Fatal("'`INSERT INTO too_many_query_args (id, value) VALUES (?, ?)`, 1, 2, 3' should return an error")
  435. }
  436. // TODO: should indicate via an error code that it is an invalid arg?
  437. }
  438. // TestNotEnoughQueryArgs tests to make sure the library correctly handles the application level bug
  439. // whereby not enough query arguments are passed to a query
  440. func TestNotEnoughQueryArgs(t *testing.T) {
  441. if *flagProto == 1 {
  442. t.Skip("atomic batches not supported. Please use Cassandra >= 2.0")
  443. }
  444. session := createSession(t)
  445. defer session.Close()
  446. if err := createTable(session, `CREATE TABLE gocql_test.not_enough_query_args (id int, cluster int, value int, primary key (id, cluster))`); err != nil {
  447. t.Fatal("create table:", err)
  448. }
  449. _, err := session.Query(`SELECT * FROM not_enough_query_args WHERE id = ? and cluster = ?`, 1).Iter().SliceMap()
  450. if err == nil {
  451. t.Fatal("'`SELECT * FROM not_enough_query_args WHERE id = ? and cluster = ?`, 1' should return an error")
  452. }
  453. batch := session.NewBatch(UnloggedBatch)
  454. batch.Query("INSERT INTO not_enough_query_args (id, cluster, value) VALUES (?, ?, ?)", 1, 2)
  455. err = session.ExecuteBatch(batch)
  456. if err == nil {
  457. t.Fatal("'`INSERT INTO not_enough_query_args (id, cluster, value) VALUES (?, ?, ?)`, 1, 2' should return an error")
  458. }
  459. }
  460. // TestCreateSessionTimeout tests to make sure the CreateSession function timeouts out correctly
  461. // and prevents an infinite loop of connection retries.
  462. func TestCreateSessionTimeout(t *testing.T) {
  463. ctx, cancel := context.WithCancel(context.Background())
  464. defer cancel()
  465. go func() {
  466. select {
  467. case <-time.After(2 * time.Second):
  468. t.Error("no startup timeout")
  469. case <-ctx.Done():
  470. }
  471. }()
  472. cluster := createCluster()
  473. cluster.Hosts = []string{"127.0.0.1:1"}
  474. session, err := cluster.CreateSession()
  475. if err == nil {
  476. session.Close()
  477. t.Fatal("expected ErrNoConnectionsStarted, but no error was returned.")
  478. }
  479. }
  480. func TestReconnection(t *testing.T) {
  481. cluster := createCluster()
  482. cluster.ReconnectInterval = 1 * time.Second
  483. session := createSessionFromCluster(cluster, t)
  484. defer session.Close()
  485. h := session.ring.allHosts()[0]
  486. session.handleNodeDown(h.Peer(), h.Port())
  487. if h.State() != NodeDown {
  488. t.Fatal("Host should be NodeDown but not.")
  489. }
  490. time.Sleep(cluster.ReconnectInterval + h.Version().nodeUpDelay() + 1*time.Second)
  491. if h.State() != NodeUp {
  492. t.Fatal("Host should be NodeUp but not. Failed to reconnect.")
  493. }
  494. }
  495. type FullName struct {
  496. FirstName string
  497. LastName string
  498. }
  499. func (n FullName) MarshalCQL(info TypeInfo) ([]byte, error) {
  500. return []byte(n.FirstName + " " + n.LastName), nil
  501. }
  502. func (n *FullName) UnmarshalCQL(info TypeInfo, data []byte) error {
  503. t := strings.SplitN(string(data), " ", 2)
  504. n.FirstName, n.LastName = t[0], t[1]
  505. return nil
  506. }
  507. func TestMapScanWithRefMap(t *testing.T) {
  508. session := createSession(t)
  509. defer session.Close()
  510. if err := createTable(session, `CREATE TABLE gocql_test.scan_map_ref_table (
  511. testtext text PRIMARY KEY,
  512. testfullname text,
  513. testint int,
  514. )`); err != nil {
  515. t.Fatal("create table:", err)
  516. }
  517. m := make(map[string]interface{})
  518. m["testtext"] = "testtext"
  519. m["testfullname"] = FullName{"John", "Doe"}
  520. m["testint"] = 100
  521. if err := session.Query(`INSERT INTO scan_map_ref_table (testtext, testfullname, testint) values (?,?,?)`,
  522. m["testtext"], m["testfullname"], m["testint"]).Exec(); err != nil {
  523. t.Fatal("insert:", err)
  524. }
  525. var testText string
  526. var testFullName FullName
  527. ret := map[string]interface{}{
  528. "testtext": &testText,
  529. "testfullname": &testFullName,
  530. // testint is not set here.
  531. }
  532. iter := session.Query(`SELECT * FROM scan_map_ref_table`).Iter()
  533. if ok := iter.MapScan(ret); !ok {
  534. t.Fatal("select:", iter.Close())
  535. } else {
  536. if ret["testtext"] != "testtext" {
  537. t.Fatal("returned testtext did not match")
  538. }
  539. f := ret["testfullname"].(FullName)
  540. if f.FirstName != "John" || f.LastName != "Doe" {
  541. t.Fatal("returned testfullname did not match")
  542. }
  543. if ret["testint"] != 100 {
  544. t.Fatal("returned testinit did not match")
  545. }
  546. }
  547. if testText != "testtext" {
  548. t.Fatal("returned testtext did not match")
  549. }
  550. if testFullName.FirstName != "John" || testFullName.LastName != "Doe" {
  551. t.Fatal("returned testfullname did not match")
  552. }
  553. }
  554. func TestMapScan(t *testing.T) {
  555. session := createSession(t)
  556. defer session.Close()
  557. if err := createTable(session, `CREATE TABLE gocql_test.scan_map_table (
  558. fullname text PRIMARY KEY,
  559. age int,
  560. address inet,
  561. )`); err != nil {
  562. t.Fatal("create table:", err)
  563. }
  564. if err := session.Query(`INSERT INTO scan_map_table (fullname, age, address) values (?,?,?)`,
  565. "Grace Hopper", 31, net.ParseIP("10.0.0.1")).Exec(); err != nil {
  566. t.Fatal("insert:", err)
  567. }
  568. if err := session.Query(`INSERT INTO scan_map_table (fullname, age, address) values (?,?,?)`,
  569. "Ada Lovelace", 30, net.ParseIP("10.0.0.2")).Exec(); err != nil {
  570. t.Fatal("insert:", err)
  571. }
  572. iter := session.Query(`SELECT * FROM scan_map_table`).Iter()
  573. // First iteration
  574. row := make(map[string]interface{})
  575. if !iter.MapScan(row) {
  576. t.Fatal("select:", iter.Close())
  577. }
  578. assertEqual(t, "fullname", "Ada Lovelace", row["fullname"])
  579. assertEqual(t, "age", 30, row["age"])
  580. assertEqual(t, "address", "10.0.0.2", row["address"])
  581. // Second iteration using a new map
  582. row = make(map[string]interface{})
  583. if !iter.MapScan(row) {
  584. t.Fatal("select:", iter.Close())
  585. }
  586. assertEqual(t, "fullname", "Grace Hopper", row["fullname"])
  587. assertEqual(t, "age", 31, row["age"])
  588. assertEqual(t, "address", "10.0.0.1", row["address"])
  589. }
  590. func TestSliceMap(t *testing.T) {
  591. session := createSession(t)
  592. defer session.Close()
  593. if err := createTable(session, `CREATE TABLE gocql_test.slice_map_table (
  594. testuuid timeuuid PRIMARY KEY,
  595. testtimestamp timestamp,
  596. testvarchar varchar,
  597. testbigint bigint,
  598. testblob blob,
  599. testbool boolean,
  600. testfloat float,
  601. testdouble double,
  602. testint int,
  603. testdecimal decimal,
  604. testlist list<text>,
  605. testset set<int>,
  606. testmap map<varchar, varchar>,
  607. testvarint varint,
  608. testinet inet
  609. )`); err != nil {
  610. t.Fatal("create table:", err)
  611. }
  612. m := make(map[string]interface{})
  613. bigInt := new(big.Int)
  614. if _, ok := bigInt.SetString("830169365738487321165427203929228", 10); !ok {
  615. t.Fatal("Failed setting bigint by string")
  616. }
  617. m["testuuid"] = TimeUUID()
  618. m["testvarchar"] = "Test VarChar"
  619. m["testbigint"] = time.Now().Unix()
  620. m["testtimestamp"] = time.Now().Truncate(time.Millisecond).UTC()
  621. m["testblob"] = []byte("test blob")
  622. m["testbool"] = true
  623. m["testfloat"] = float32(4.564)
  624. m["testdouble"] = float64(4.815162342)
  625. m["testint"] = 2343
  626. m["testdecimal"] = inf.NewDec(100, 0)
  627. m["testlist"] = []string{"quux", "foo", "bar", "baz", "quux"}
  628. m["testset"] = []int{1, 2, 3, 4, 5, 6, 7, 8, 9}
  629. m["testmap"] = map[string]string{"field1": "val1", "field2": "val2", "field3": "val3"}
  630. m["testvarint"] = bigInt
  631. m["testinet"] = "213.212.2.19"
  632. sliceMap := []map[string]interface{}{m}
  633. if err := session.Query(`INSERT INTO slice_map_table (testuuid, testtimestamp, testvarchar, testbigint, testblob, testbool, testfloat, testdouble, testint, testdecimal, testlist, testset, testmap, testvarint, testinet) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)`,
  634. m["testuuid"], m["testtimestamp"], m["testvarchar"], m["testbigint"], m["testblob"], m["testbool"], m["testfloat"], m["testdouble"], m["testint"], m["testdecimal"], m["testlist"], m["testset"], m["testmap"], m["testvarint"], m["testinet"]).Exec(); err != nil {
  635. t.Fatal("insert:", err)
  636. }
  637. if returned, retErr := session.Query(`SELECT * FROM slice_map_table`).Iter().SliceMap(); retErr != nil {
  638. t.Fatal("select:", retErr)
  639. } else {
  640. matchSliceMap(t, sliceMap, returned[0])
  641. }
  642. // Test for Iter.MapScan()
  643. {
  644. testMap := make(map[string]interface{})
  645. if !session.Query(`SELECT * FROM slice_map_table`).Iter().MapScan(testMap) {
  646. t.Fatal("MapScan failed to work with one row")
  647. }
  648. matchSliceMap(t, sliceMap, testMap)
  649. }
  650. // Test for Query.MapScan()
  651. {
  652. testMap := make(map[string]interface{})
  653. if session.Query(`SELECT * FROM slice_map_table`).MapScan(testMap) != nil {
  654. t.Fatal("MapScan failed to work with one row")
  655. }
  656. matchSliceMap(t, sliceMap, testMap)
  657. }
  658. }
  659. func matchSliceMap(t *testing.T, sliceMap []map[string]interface{}, testMap map[string]interface{}) {
  660. if sliceMap[0]["testuuid"] != testMap["testuuid"] {
  661. t.Fatal("returned testuuid did not match")
  662. }
  663. if sliceMap[0]["testtimestamp"] != testMap["testtimestamp"] {
  664. t.Fatal("returned testtimestamp did not match")
  665. }
  666. if sliceMap[0]["testvarchar"] != testMap["testvarchar"] {
  667. t.Fatal("returned testvarchar did not match")
  668. }
  669. if sliceMap[0]["testbigint"] != testMap["testbigint"] {
  670. t.Fatal("returned testbigint did not match")
  671. }
  672. if !reflect.DeepEqual(sliceMap[0]["testblob"], testMap["testblob"]) {
  673. t.Fatal("returned testblob did not match")
  674. }
  675. if sliceMap[0]["testbool"] != testMap["testbool"] {
  676. t.Fatal("returned testbool did not match")
  677. }
  678. if sliceMap[0]["testfloat"] != testMap["testfloat"] {
  679. t.Fatal("returned testfloat did not match")
  680. }
  681. if sliceMap[0]["testdouble"] != testMap["testdouble"] {
  682. t.Fatal("returned testdouble did not match")
  683. }
  684. if sliceMap[0]["testinet"] != testMap["testinet"] {
  685. t.Fatal("returned testinet did not match")
  686. }
  687. expectedDecimal := sliceMap[0]["testdecimal"].(*inf.Dec)
  688. returnedDecimal := testMap["testdecimal"].(*inf.Dec)
  689. if expectedDecimal.Cmp(returnedDecimal) != 0 {
  690. t.Fatal("returned testdecimal did not match")
  691. }
  692. if !reflect.DeepEqual(sliceMap[0]["testlist"], testMap["testlist"]) {
  693. t.Fatal("returned testlist did not match")
  694. }
  695. if !reflect.DeepEqual(sliceMap[0]["testset"], testMap["testset"]) {
  696. t.Fatal("returned testset did not match")
  697. }
  698. if !reflect.DeepEqual(sliceMap[0]["testmap"], testMap["testmap"]) {
  699. t.Fatal("returned testmap did not match")
  700. }
  701. if sliceMap[0]["testint"] != testMap["testint"] {
  702. t.Fatal("returned testint did not match")
  703. }
  704. }
  705. func TestSmallInt(t *testing.T) {
  706. if *flagProto < protoVersion4 {
  707. t.Skip("smallint is only supported in cassandra 2.2+")
  708. }
  709. session := createSession(t)
  710. defer session.Close()
  711. if err := createTable(session, `CREATE TABLE gocql_test.smallint_table (
  712. testsmallint smallint PRIMARY KEY,
  713. )`); err != nil {
  714. t.Fatal("create table:", err)
  715. }
  716. m := make(map[string]interface{})
  717. m["testsmallint"] = int16(2)
  718. sliceMap := []map[string]interface{}{m}
  719. if err := session.Query(`INSERT INTO smallint_table (testsmallint) VALUES (?)`,
  720. m["testsmallint"]).Exec(); err != nil {
  721. t.Fatal("insert:", err)
  722. }
  723. if returned, retErr := session.Query(`SELECT * FROM smallint_table`).Iter().SliceMap(); retErr != nil {
  724. t.Fatal("select:", retErr)
  725. } else {
  726. if sliceMap[0]["testsmallint"] != returned[0]["testsmallint"] {
  727. t.Fatal("returned testsmallint did not match")
  728. }
  729. }
  730. }
  731. func TestScanWithNilArguments(t *testing.T) {
  732. session := createSession(t)
  733. defer session.Close()
  734. if err := createTable(session, `CREATE TABLE gocql_test.scan_with_nil_arguments (
  735. foo varchar,
  736. bar int,
  737. PRIMARY KEY (foo, bar)
  738. )`); err != nil {
  739. t.Fatal("create:", err)
  740. }
  741. for i := 1; i <= 20; i++ {
  742. if err := session.Query("INSERT INTO scan_with_nil_arguments (foo, bar) VALUES (?, ?)",
  743. "squares", i*i).Exec(); err != nil {
  744. t.Fatal("insert:", err)
  745. }
  746. }
  747. iter := session.Query("SELECT * FROM scan_with_nil_arguments WHERE foo = ?", "squares").Iter()
  748. var n int
  749. count := 0
  750. for iter.Scan(nil, &n) {
  751. count += n
  752. }
  753. if err := iter.Close(); err != nil {
  754. t.Fatal("close:", err)
  755. }
  756. if count != 2870 {
  757. t.Fatalf("expected %d, got %d", 2870, count)
  758. }
  759. }
  760. func TestScanCASWithNilArguments(t *testing.T) {
  761. if *flagProto == 1 {
  762. t.Skip("lightweight transactions not supported. Please use Cassandra >= 2.0")
  763. }
  764. session := createSession(t)
  765. defer session.Close()
  766. if err := createTable(session, `CREATE TABLE gocql_test.scan_cas_with_nil_arguments (
  767. foo varchar,
  768. bar varchar,
  769. PRIMARY KEY (foo, bar)
  770. )`); err != nil {
  771. t.Fatal("create:", err)
  772. }
  773. foo := "baz"
  774. var cas string
  775. if applied, err := session.Query(`INSERT INTO scan_cas_with_nil_arguments (foo, bar)
  776. VALUES (?, ?) IF NOT EXISTS`,
  777. foo, foo).ScanCAS(nil, nil); err != nil {
  778. t.Fatal("insert:", err)
  779. } else if !applied {
  780. t.Fatal("insert should have been applied")
  781. }
  782. if applied, err := session.Query(`INSERT INTO scan_cas_with_nil_arguments (foo, bar)
  783. VALUES (?, ?) IF NOT EXISTS`,
  784. foo, foo).ScanCAS(&cas, nil); err != nil {
  785. t.Fatal("insert:", err)
  786. } else if applied {
  787. t.Fatal("insert should not have been applied")
  788. } else if foo != cas {
  789. t.Fatalf("expected %v but got %v", foo, cas)
  790. }
  791. if applied, err := session.Query(`INSERT INTO scan_cas_with_nil_arguments (foo, bar)
  792. VALUES (?, ?) IF NOT EXISTS`,
  793. foo, foo).ScanCAS(nil, &cas); err != nil {
  794. t.Fatal("insert:", err)
  795. } else if applied {
  796. t.Fatal("insert should not have been applied")
  797. } else if foo != cas {
  798. t.Fatalf("expected %v but got %v", foo, cas)
  799. }
  800. }
  801. func TestRebindQueryInfo(t *testing.T) {
  802. session := createSession(t)
  803. defer session.Close()
  804. if err := createTable(session, "CREATE TABLE gocql_test.rebind_query (id int, value text, PRIMARY KEY (id))"); err != nil {
  805. t.Fatalf("failed to create table with error '%v'", err)
  806. }
  807. if err := session.Query("INSERT INTO rebind_query (id, value) VALUES (?, ?)", 23, "quux").Exec(); err != nil {
  808. t.Fatalf("insert into rebind_query failed, err '%v'", err)
  809. }
  810. if err := session.Query("INSERT INTO rebind_query (id, value) VALUES (?, ?)", 24, "w00t").Exec(); err != nil {
  811. t.Fatalf("insert into rebind_query failed, err '%v'", err)
  812. }
  813. q := session.Query("SELECT value FROM rebind_query WHERE ID = ?")
  814. q.Bind(23)
  815. iter := q.Iter()
  816. var value string
  817. for iter.Scan(&value) {
  818. }
  819. if value != "quux" {
  820. t.Fatalf("expected %v but got %v", "quux", value)
  821. }
  822. q.Bind(24)
  823. iter = q.Iter()
  824. for iter.Scan(&value) {
  825. }
  826. if value != "w00t" {
  827. t.Fatalf("expected %v but got %v", "quux", value)
  828. }
  829. }
  830. //TestStaticQueryInfo makes sure that the application can manually bind query parameters using the simplest possible static binding strategy
  831. func TestStaticQueryInfo(t *testing.T) {
  832. session := createSession(t)
  833. defer session.Close()
  834. if err := createTable(session, "CREATE TABLE gocql_test.static_query_info (id int, value text, PRIMARY KEY (id))"); err != nil {
  835. t.Fatalf("failed to create table with error '%v'", err)
  836. }
  837. if err := session.Query("INSERT INTO static_query_info (id, value) VALUES (?, ?)", 113, "foo").Exec(); err != nil {
  838. t.Fatalf("insert into static_query_info failed, err '%v'", err)
  839. }
  840. autobinder := func(q *QueryInfo) ([]interface{}, error) {
  841. values := make([]interface{}, 1)
  842. values[0] = 113
  843. return values, nil
  844. }
  845. qry := session.Bind("SELECT id, value FROM static_query_info WHERE id = ?", autobinder)
  846. if err := qry.Exec(); err != nil {
  847. t.Fatalf("expose query info failed, error '%v'", err)
  848. }
  849. iter := qry.Iter()
  850. var id int
  851. var value string
  852. iter.Scan(&id, &value)
  853. if err := iter.Close(); err != nil {
  854. t.Fatalf("query with exposed info failed, err '%v'", err)
  855. }
  856. if value != "foo" {
  857. t.Fatalf("Expected value %s, but got %s", "foo", value)
  858. }
  859. }
  860. type ClusteredKeyValue struct {
  861. Id int
  862. Cluster int
  863. Value string
  864. }
  865. func (kv *ClusteredKeyValue) Bind(q *QueryInfo) ([]interface{}, error) {
  866. values := make([]interface{}, len(q.Args))
  867. for i, info := range q.Args {
  868. fieldName := upcaseInitial(info.Name)
  869. value := reflect.ValueOf(kv)
  870. field := reflect.Indirect(value).FieldByName(fieldName)
  871. values[i] = field.Addr().Interface()
  872. }
  873. return values, nil
  874. }
  875. func upcaseInitial(str string) string {
  876. for i, v := range str {
  877. return string(unicode.ToUpper(v)) + str[i+1:]
  878. }
  879. return ""
  880. }
  881. //TestBoundQueryInfo makes sure that the application can manually bind query parameters using the query meta data supplied at runtime
  882. func TestBoundQueryInfo(t *testing.T) {
  883. session := createSession(t)
  884. defer session.Close()
  885. if err := createTable(session, "CREATE TABLE gocql_test.clustered_query_info (id int, cluster int, value text, PRIMARY KEY (id, cluster))"); err != nil {
  886. t.Fatalf("failed to create table with error '%v'", err)
  887. }
  888. write := &ClusteredKeyValue{Id: 200, Cluster: 300, Value: "baz"}
  889. insert := session.Bind("INSERT INTO clustered_query_info (id, cluster, value) VALUES (?, ?,?)", write.Bind)
  890. if err := insert.Exec(); err != nil {
  891. t.Fatalf("insert into clustered_query_info failed, err '%v'", err)
  892. }
  893. read := &ClusteredKeyValue{Id: 200, Cluster: 300}
  894. qry := session.Bind("SELECT id, cluster, value FROM clustered_query_info WHERE id = ? and cluster = ?", read.Bind)
  895. iter := qry.Iter()
  896. var id, cluster int
  897. var value string
  898. iter.Scan(&id, &cluster, &value)
  899. if err := iter.Close(); err != nil {
  900. t.Fatalf("query with clustered_query_info info failed, err '%v'", err)
  901. }
  902. if value != "baz" {
  903. t.Fatalf("Expected value %s, but got %s", "baz", value)
  904. }
  905. }
  906. //TestBatchQueryInfo makes sure that the application can manually bind query parameters when executing in a batch
  907. func TestBatchQueryInfo(t *testing.T) {
  908. if *flagProto == 1 {
  909. t.Skip("atomic batches not supported. Please use Cassandra >= 2.0")
  910. }
  911. session := createSession(t)
  912. defer session.Close()
  913. if err := createTable(session, "CREATE TABLE gocql_test.batch_query_info (id int, cluster int, value text, PRIMARY KEY (id, cluster))"); err != nil {
  914. t.Fatalf("failed to create table with error '%v'", err)
  915. }
  916. write := func(q *QueryInfo) ([]interface{}, error) {
  917. values := make([]interface{}, 3)
  918. values[0] = 4000
  919. values[1] = 5000
  920. values[2] = "bar"
  921. return values, nil
  922. }
  923. batch := session.NewBatch(LoggedBatch)
  924. batch.Bind("INSERT INTO batch_query_info (id, cluster, value) VALUES (?, ?,?)", write)
  925. if err := session.ExecuteBatch(batch); err != nil {
  926. t.Fatalf("batch insert into batch_query_info failed, err '%v'", err)
  927. }
  928. read := func(q *QueryInfo) ([]interface{}, error) {
  929. values := make([]interface{}, 2)
  930. values[0] = 4000
  931. values[1] = 5000
  932. return values, nil
  933. }
  934. qry := session.Bind("SELECT id, cluster, value FROM batch_query_info WHERE id = ? and cluster = ?", read)
  935. iter := qry.Iter()
  936. var id, cluster int
  937. var value string
  938. iter.Scan(&id, &cluster, &value)
  939. if err := iter.Close(); err != nil {
  940. t.Fatalf("query with batch_query_info info failed, err '%v'", err)
  941. }
  942. if value != "bar" {
  943. t.Fatalf("Expected value %s, but got %s", "bar", value)
  944. }
  945. }
  946. func getRandomConn(t *testing.T, session *Session) *Conn {
  947. conn := session.getConn()
  948. if conn == nil {
  949. t.Fatal("unable to get a connection")
  950. }
  951. return conn
  952. }
  953. func injectInvalidPreparedStatement(t *testing.T, session *Session, table string) (string, *Conn) {
  954. if err := createTable(session, `CREATE TABLE gocql_test.`+table+` (
  955. foo varchar,
  956. bar int,
  957. PRIMARY KEY (foo, bar)
  958. )`); err != nil {
  959. t.Fatal("create:", err)
  960. }
  961. stmt := "INSERT INTO " + table + " (foo, bar) VALUES (?, 7)"
  962. conn := getRandomConn(t, session)
  963. flight := new(inflightPrepare)
  964. key := session.stmtsLRU.keyFor(conn.addr, "", stmt)
  965. session.stmtsLRU.add(key, flight)
  966. flight.preparedStatment = &preparedStatment{
  967. id: []byte{'f', 'o', 'o', 'b', 'a', 'r'},
  968. request: preparedMetadata{
  969. resultMetadata: resultMetadata{
  970. colCount: 1,
  971. actualColCount: 1,
  972. columns: []ColumnInfo{
  973. {
  974. Keyspace: "gocql_test",
  975. Table: table,
  976. Name: "foo",
  977. TypeInfo: NativeType{
  978. typ: TypeVarchar,
  979. },
  980. },
  981. },
  982. },
  983. },
  984. }
  985. return stmt, conn
  986. }
  987. func TestPrepare_MissingSchemaPrepare(t *testing.T) {
  988. s := createSession(t)
  989. conn := getRandomConn(t, s)
  990. defer s.Close()
  991. insertQry := &Query{stmt: "INSERT INTO invalidschemaprep (val) VALUES (?)", values: []interface{}{5}, cons: s.cons,
  992. session: s, pageSize: s.pageSize, trace: s.trace,
  993. prefetch: s.prefetch, rt: s.cfg.RetryPolicy}
  994. if err := conn.executeQuery(insertQry).err; err == nil {
  995. t.Fatal("expected error, but got nil.")
  996. }
  997. if err := createTable(s, "CREATE TABLE gocql_test.invalidschemaprep (val int, PRIMARY KEY (val))"); err != nil {
  998. t.Fatal("create table:", err)
  999. }
  1000. if err := conn.executeQuery(insertQry).err; err != nil {
  1001. t.Fatal(err) // unconfigured columnfamily
  1002. }
  1003. }
  1004. func TestPrepare_ReprepareStatement(t *testing.T) {
  1005. session := createSession(t)
  1006. defer session.Close()
  1007. stmt, conn := injectInvalidPreparedStatement(t, session, "test_reprepare_statement")
  1008. query := session.Query(stmt, "bar")
  1009. if err := conn.executeQuery(query).Close(); err != nil {
  1010. t.Fatalf("Failed to execute query for reprepare statement: %v", err)
  1011. }
  1012. }
  1013. func TestPrepare_ReprepareBatch(t *testing.T) {
  1014. if *flagProto == 1 {
  1015. t.Skip("atomic batches not supported. Please use Cassandra >= 2.0")
  1016. }
  1017. session := createSession(t)
  1018. defer session.Close()
  1019. stmt, conn := injectInvalidPreparedStatement(t, session, "test_reprepare_statement_batch")
  1020. batch := session.NewBatch(UnloggedBatch)
  1021. batch.Query(stmt, "bar")
  1022. if err := conn.executeBatch(batch).Close(); err != nil {
  1023. t.Fatalf("Failed to execute query for reprepare statement: %v", err)
  1024. }
  1025. }
  1026. func TestQueryInfo(t *testing.T) {
  1027. session := createSession(t)
  1028. defer session.Close()
  1029. conn := getRandomConn(t, session)
  1030. info, err := conn.prepareStatement(context.Background(), "SELECT release_version, host_id FROM system.local WHERE key = ?", nil)
  1031. if err != nil {
  1032. t.Fatalf("Failed to execute query for preparing statement: %v", err)
  1033. }
  1034. if x := len(info.request.columns); x != 1 {
  1035. t.Fatalf("Was not expecting meta data for %d query arguments, but got %d\n", 1, x)
  1036. }
  1037. if *flagProto > 1 {
  1038. if x := len(info.response.columns); x != 2 {
  1039. t.Fatalf("Was not expecting meta data for %d result columns, but got %d\n", 2, x)
  1040. }
  1041. }
  1042. }
  1043. //TestPreparedCacheEviction will make sure that the cache size is maintained
  1044. func TestPrepare_PreparedCacheEviction(t *testing.T) {
  1045. const maxPrepared = 4
  1046. host := clusterHosts[0]
  1047. cluster := createCluster()
  1048. cluster.MaxPreparedStmts = maxPrepared
  1049. cluster.Events.DisableSchemaEvents = true
  1050. cluster.Hosts = []string{host}
  1051. cluster.HostFilter = WhiteListHostFilter(host)
  1052. session := createSessionFromCluster(cluster, t)
  1053. defer session.Close()
  1054. if err := createTable(session, "CREATE TABLE gocql_test.prepcachetest (id int,mod int,PRIMARY KEY (id))"); err != nil {
  1055. t.Fatalf("failed to create table with error '%v'", err)
  1056. }
  1057. // clear the cache
  1058. session.stmtsLRU.clear()
  1059. //Fill the table
  1060. for i := 0; i < 2; i++ {
  1061. if err := session.Query("INSERT INTO prepcachetest (id,mod) VALUES (?, ?)", i, 10000%(i+1)).Exec(); err != nil {
  1062. t.Fatalf("insert into prepcachetest failed, err '%v'", err)
  1063. }
  1064. }
  1065. //Populate the prepared statement cache with select statements
  1066. var id, mod int
  1067. for i := 0; i < 2; i++ {
  1068. err := session.Query("SELECT id,mod FROM prepcachetest WHERE id = "+strconv.FormatInt(int64(i), 10)).Scan(&id, &mod)
  1069. if err != nil {
  1070. t.Fatalf("select from prepcachetest failed, error '%v'", err)
  1071. }
  1072. }
  1073. //generate an update statement to test they are prepared
  1074. err := session.Query("UPDATE prepcachetest SET mod = ? WHERE id = ?", 1, 11).Exec()
  1075. if err != nil {
  1076. t.Fatalf("update prepcachetest failed, error '%v'", err)
  1077. }
  1078. //generate a delete statement to test they are prepared
  1079. err = session.Query("DELETE FROM prepcachetest WHERE id = ?", 1).Exec()
  1080. if err != nil {
  1081. t.Fatalf("delete from prepcachetest failed, error '%v'", err)
  1082. }
  1083. //generate an insert statement to test they are prepared
  1084. err = session.Query("INSERT INTO prepcachetest (id,mod) VALUES (?, ?)", 3, 11).Exec()
  1085. if err != nil {
  1086. t.Fatalf("insert into prepcachetest failed, error '%v'", err)
  1087. }
  1088. session.stmtsLRU.mu.Lock()
  1089. defer session.stmtsLRU.mu.Unlock()
  1090. //Make sure the cache size is maintained
  1091. if session.stmtsLRU.lru.Len() != session.stmtsLRU.lru.MaxEntries {
  1092. t.Fatalf("expected cache size of %v, got %v", session.stmtsLRU.lru.MaxEntries, session.stmtsLRU.lru.Len())
  1093. }
  1094. // Walk through all the configured hosts and test cache retention and eviction
  1095. for _, host := range session.cfg.Hosts {
  1096. _, ok := session.stmtsLRU.lru.Get(session.stmtsLRU.keyFor(host+":9042", session.cfg.Keyspace, "SELECT id,mod FROM prepcachetest WHERE id = 0"))
  1097. if ok {
  1098. t.Errorf("expected first select to be purged but was in cache for host=%q", host)
  1099. }
  1100. _, ok = session.stmtsLRU.lru.Get(session.stmtsLRU.keyFor(host+":9042", session.cfg.Keyspace, "SELECT id,mod FROM prepcachetest WHERE id = 1"))
  1101. if !ok {
  1102. t.Errorf("exepected second select to be in cache for host=%q", host)
  1103. }
  1104. _, ok = session.stmtsLRU.lru.Get(session.stmtsLRU.keyFor(host+":9042", session.cfg.Keyspace, "INSERT INTO prepcachetest (id,mod) VALUES (?, ?)"))
  1105. if !ok {
  1106. t.Errorf("expected insert to be in cache for host=%q", host)
  1107. }
  1108. _, ok = session.stmtsLRU.lru.Get(session.stmtsLRU.keyFor(host+":9042", session.cfg.Keyspace, "UPDATE prepcachetest SET mod = ? WHERE id = ?"))
  1109. if !ok {
  1110. t.Errorf("expected update to be in cached for host=%q", host)
  1111. }
  1112. _, ok = session.stmtsLRU.lru.Get(session.stmtsLRU.keyFor(host+":9042", session.cfg.Keyspace, "DELETE FROM prepcachetest WHERE id = ?"))
  1113. if !ok {
  1114. t.Errorf("expected delete to be cached for host=%q", host)
  1115. }
  1116. }
  1117. }
  1118. func TestPrepare_PreparedCacheKey(t *testing.T) {
  1119. session := createSession(t)
  1120. defer session.Close()
  1121. // create a second keyspace
  1122. cluster2 := createCluster()
  1123. createKeyspace(t, cluster2, "gocql_test2")
  1124. cluster2.Keyspace = "gocql_test2"
  1125. session2, err := cluster2.CreateSession()
  1126. if err != nil {
  1127. t.Fatal("create session:", err)
  1128. }
  1129. defer session2.Close()
  1130. // both keyspaces have a table named "test_stmt_cache_key"
  1131. if err := createTable(session, "CREATE TABLE gocql_test.test_stmt_cache_key (id varchar primary key, field varchar)"); err != nil {
  1132. t.Fatal("create table:", err)
  1133. }
  1134. if err := createTable(session2, "CREATE TABLE gocql_test2.test_stmt_cache_key (id varchar primary key, field varchar)"); err != nil {
  1135. t.Fatal("create table:", err)
  1136. }
  1137. // both tables have a single row with the same partition key but different column value
  1138. if err = session.Query(`INSERT INTO test_stmt_cache_key (id, field) VALUES (?, ?)`, "key", "one").Exec(); err != nil {
  1139. t.Fatal("insert:", err)
  1140. }
  1141. if err = session2.Query(`INSERT INTO test_stmt_cache_key (id, field) VALUES (?, ?)`, "key", "two").Exec(); err != nil {
  1142. t.Fatal("insert:", err)
  1143. }
  1144. // should be able to see different values in each keyspace
  1145. var value string
  1146. if err = session.Query("SELECT field FROM test_stmt_cache_key WHERE id = ?", "key").Scan(&value); err != nil {
  1147. t.Fatal("select:", err)
  1148. }
  1149. if value != "one" {
  1150. t.Errorf("Expected one, got %s", value)
  1151. }
  1152. if err = session2.Query("SELECT field FROM test_stmt_cache_key WHERE id = ?", "key").Scan(&value); err != nil {
  1153. t.Fatal("select:", err)
  1154. }
  1155. if value != "two" {
  1156. t.Errorf("Expected two, got %s", value)
  1157. }
  1158. }
  1159. //TestMarshalFloat64Ptr tests to see that a pointer to a float64 is marshalled correctly.
  1160. func TestMarshalFloat64Ptr(t *testing.T) {
  1161. session := createSession(t)
  1162. defer session.Close()
  1163. if err := createTable(session, "CREATE TABLE gocql_test.float_test (id double, test double, primary key (id))"); err != nil {
  1164. t.Fatal("create table:", err)
  1165. }
  1166. testNum := float64(7500)
  1167. if err := session.Query(`INSERT INTO float_test (id,test) VALUES (?,?)`, float64(7500.00), &testNum).Exec(); err != nil {
  1168. t.Fatal("insert float64:", err)
  1169. }
  1170. }
  1171. //TestMarshalInet tests to see that a pointer to a float64 is marshalled correctly.
  1172. func TestMarshalInet(t *testing.T) {
  1173. session := createSession(t)
  1174. defer session.Close()
  1175. if err := createTable(session, "CREATE TABLE gocql_test.inet_test (ip inet, name text, primary key (ip))"); err != nil {
  1176. t.Fatal("create table:", err)
  1177. }
  1178. stringIp := "123.34.45.56"
  1179. if err := session.Query(`INSERT INTO inet_test (ip,name) VALUES (?,?)`, stringIp, "Test IP 1").Exec(); err != nil {
  1180. t.Fatal("insert string inet:", err)
  1181. }
  1182. var stringResult string
  1183. if err := session.Query("SELECT ip FROM inet_test").Scan(&stringResult); err != nil {
  1184. t.Fatalf("select for string from inet_test 1 failed: %v", err)
  1185. }
  1186. if stringResult != stringIp {
  1187. t.Errorf("Expected %s, was %s", stringIp, stringResult)
  1188. }
  1189. var ipResult net.IP
  1190. if err := session.Query("SELECT ip FROM inet_test").Scan(&ipResult); err != nil {
  1191. t.Fatalf("select for net.IP from inet_test 1 failed: %v", err)
  1192. }
  1193. if ipResult.String() != stringIp {
  1194. t.Errorf("Expected %s, was %s", stringIp, ipResult.String())
  1195. }
  1196. if err := session.Query(`DELETE FROM inet_test WHERE ip = ?`, stringIp).Exec(); err != nil {
  1197. t.Fatal("delete inet table:", err)
  1198. }
  1199. netIp := net.ParseIP("222.43.54.65")
  1200. if err := session.Query(`INSERT INTO inet_test (ip,name) VALUES (?,?)`, netIp, "Test IP 2").Exec(); err != nil {
  1201. t.Fatal("insert netIp inet:", err)
  1202. }
  1203. if err := session.Query("SELECT ip FROM inet_test").Scan(&stringResult); err != nil {
  1204. t.Fatalf("select for string from inet_test 2 failed: %v", err)
  1205. }
  1206. if stringResult != netIp.String() {
  1207. t.Errorf("Expected %s, was %s", netIp.String(), stringResult)
  1208. }
  1209. if err := session.Query("SELECT ip FROM inet_test").Scan(&ipResult); err != nil {
  1210. t.Fatalf("select for net.IP from inet_test 2 failed: %v", err)
  1211. }
  1212. if ipResult.String() != netIp.String() {
  1213. t.Errorf("Expected %s, was %s", netIp.String(), ipResult.String())
  1214. }
  1215. }
  1216. func TestVarint(t *testing.T) {
  1217. session := createSession(t)
  1218. defer session.Close()
  1219. if err := createTable(session, "CREATE TABLE gocql_test.varint_test (id varchar, test varint, test2 varint, primary key (id))"); err != nil {
  1220. t.Fatalf("failed to create table with error '%v'", err)
  1221. }
  1222. if err := session.Query(`INSERT INTO varint_test (id, test) VALUES (?, ?)`, "id", 0).Exec(); err != nil {
  1223. t.Fatalf("insert varint: %v", err)
  1224. }
  1225. var result int
  1226. if err := session.Query("SELECT test FROM varint_test").Scan(&result); err != nil {
  1227. t.Fatalf("select from varint_test failed: %v", err)
  1228. }
  1229. if result != 0 {
  1230. t.Errorf("Expected 0, was %d", result)
  1231. }
  1232. if err := session.Query(`INSERT INTO varint_test (id, test) VALUES (?, ?)`, "id", -1).Exec(); err != nil {
  1233. t.Fatalf("insert varint: %v", err)
  1234. }
  1235. if err := session.Query("SELECT test FROM varint_test").Scan(&result); err != nil {
  1236. t.Fatalf("select from varint_test failed: %v", err)
  1237. }
  1238. if result != -1 {
  1239. t.Errorf("Expected -1, was %d", result)
  1240. }
  1241. if err := session.Query(`INSERT INTO varint_test (id, test) VALUES (?, ?)`, "id", nil).Exec(); err != nil {
  1242. t.Fatalf("insert varint: %v", err)
  1243. }
  1244. if err := session.Query("SELECT test FROM varint_test").Scan(&result); err != nil {
  1245. t.Fatalf("select from varint_test failed: %v", err)
  1246. }
  1247. if result != 0 {
  1248. t.Errorf("Expected 0, was %d", result)
  1249. }
  1250. var nullableResult *int
  1251. if err := session.Query("SELECT test FROM varint_test").Scan(&nullableResult); err != nil {
  1252. t.Fatalf("select from varint_test failed: %v", err)
  1253. }
  1254. if nullableResult != nil {
  1255. t.Errorf("Expected nil, was %d", nullableResult)
  1256. }
  1257. if err := session.Query(`INSERT INTO varint_test (id, test) VALUES (?, ?)`, "id", int64(math.MaxInt32)+1).Exec(); err != nil {
  1258. t.Fatalf("insert varint: %v", err)
  1259. }
  1260. var result64 int64
  1261. if err := session.Query("SELECT test FROM varint_test").Scan(&result64); err != nil {
  1262. t.Fatalf("select from varint_test failed: %v", err)
  1263. }
  1264. if result64 != int64(math.MaxInt32)+1 {
  1265. t.Errorf("Expected %d, was %d", int64(math.MaxInt32)+1, result64)
  1266. }
  1267. biggie := new(big.Int)
  1268. biggie.SetString("36893488147419103232", 10) // > 2**64
  1269. if err := session.Query(`INSERT INTO varint_test (id, test) VALUES (?, ?)`, "id", biggie).Exec(); err != nil {
  1270. t.Fatalf("insert varint: %v", err)
  1271. }
  1272. resultBig := new(big.Int)
  1273. if err := session.Query("SELECT test FROM varint_test").Scan(resultBig); err != nil {
  1274. t.Fatalf("select from varint_test failed: %v", err)
  1275. }
  1276. if resultBig.String() != biggie.String() {
  1277. t.Errorf("Expected %s, was %s", biggie.String(), resultBig.String())
  1278. }
  1279. err := session.Query("SELECT test FROM varint_test").Scan(&result64)
  1280. if err == nil || strings.Index(err.Error(), "out of range") == -1 {
  1281. t.Errorf("expected out of range error since value is too big for int64")
  1282. }
  1283. // value not set in cassandra, leave bind variable empty
  1284. resultBig = new(big.Int)
  1285. if err := session.Query("SELECT test2 FROM varint_test").Scan(resultBig); err != nil {
  1286. t.Fatalf("select from varint_test failed: %v", err)
  1287. }
  1288. if resultBig.Int64() != 0 {
  1289. t.Errorf("Expected %s, was %s", biggie.String(), resultBig.String())
  1290. }
  1291. // can use double pointer to explicitly detect value is not set in cassandra
  1292. if err := session.Query("SELECT test2 FROM varint_test").Scan(&resultBig); err != nil {
  1293. t.Fatalf("select from varint_test failed: %v", err)
  1294. }
  1295. if resultBig != nil {
  1296. t.Errorf("Expected %v, was %v", nil, *resultBig)
  1297. }
  1298. }
  1299. //TestQueryStats confirms that the stats are returning valid data. Accuracy may be questionable.
  1300. func TestQueryStats(t *testing.T) {
  1301. session := createSession(t)
  1302. defer session.Close()
  1303. qry := session.Query("SELECT * FROM system.peers")
  1304. if err := qry.Exec(); err != nil {
  1305. t.Fatalf("query failed. %v", err)
  1306. } else {
  1307. if qry.Attempts() < 1 {
  1308. t.Fatal("expected at least 1 attempt, but got 0")
  1309. }
  1310. if qry.Latency() <= 0 {
  1311. t.Fatalf("expected latency to be greater than 0, but got %v instead.", qry.Latency())
  1312. }
  1313. }
  1314. }
  1315. //TestBatchStats confirms that the stats are returning valid data. Accuracy may be questionable.
  1316. func TestBatchStats(t *testing.T) {
  1317. if *flagProto == 1 {
  1318. t.Skip("atomic batches not supported. Please use Cassandra >= 2.0")
  1319. }
  1320. session := createSession(t)
  1321. defer session.Close()
  1322. if err := createTable(session, "CREATE TABLE gocql_test.batchStats (id int, PRIMARY KEY (id))"); err != nil {
  1323. t.Fatalf("failed to create table with error '%v'", err)
  1324. }
  1325. b := session.NewBatch(LoggedBatch)
  1326. b.Query("INSERT INTO batchStats (id) VALUES (?)", 1)
  1327. b.Query("INSERT INTO batchStats (id) VALUES (?)", 2)
  1328. if err := session.ExecuteBatch(b); err != nil {
  1329. t.Fatalf("query failed. %v", err)
  1330. } else {
  1331. if b.Attempts() < 1 {
  1332. t.Fatal("expected at least 1 attempt, but got 0")
  1333. }
  1334. if b.Latency() <= 0 {
  1335. t.Fatalf("expected latency to be greater than 0, but got %v instead.", b.Latency())
  1336. }
  1337. }
  1338. }
  1339. //TestNilInQuery tests to see that a nil value passed to a query is handled by Cassandra
  1340. //TODO validate the nil value by reading back the nil. Need to fix Unmarshalling.
  1341. func TestNilInQuery(t *testing.T) {
  1342. session := createSession(t)
  1343. defer session.Close()
  1344. if err := createTable(session, "CREATE TABLE gocql_test.testNilInsert (id int, count int, PRIMARY KEY (id))"); err != nil {
  1345. t.Fatalf("failed to create table with error '%v'", err)
  1346. }
  1347. if err := session.Query("INSERT INTO testNilInsert (id,count) VALUES (?,?)", 1, nil).Exec(); err != nil {
  1348. t.Fatalf("failed to insert with err: %v", err)
  1349. }
  1350. var id int
  1351. if err := session.Query("SELECT id FROM testNilInsert").Scan(&id); err != nil {
  1352. t.Fatalf("failed to select with err: %v", err)
  1353. } else if id != 1 {
  1354. t.Fatalf("expected id to be 1, got %v", id)
  1355. }
  1356. }
  1357. // Don't initialize time.Time bind variable if cassandra timestamp column is empty
  1358. func TestEmptyTimestamp(t *testing.T) {
  1359. session := createSession(t)
  1360. defer session.Close()
  1361. if err := createTable(session, "CREATE TABLE gocql_test.test_empty_timestamp (id int, time timestamp, num int, PRIMARY KEY (id))"); err != nil {
  1362. t.Fatalf("failed to create table with error '%v'", err)
  1363. }
  1364. if err := session.Query("INSERT INTO test_empty_timestamp (id, num) VALUES (?,?)", 1, 561).Exec(); err != nil {
  1365. t.Fatalf("failed to insert with err: %v", err)
  1366. }
  1367. var timeVal time.Time
  1368. if err := session.Query("SELECT time FROM test_empty_timestamp where id = ?", 1).Scan(&timeVal); err != nil {
  1369. t.Fatalf("failed to select with err: %v", err)
  1370. }
  1371. if !timeVal.IsZero() {
  1372. t.Errorf("time.Time bind variable should still be empty (was %s)", timeVal)
  1373. }
  1374. }
  1375. // Integration test of just querying for data from the system.schema_keyspace table where the keyspace DOES exist.
  1376. func TestGetKeyspaceMetadata(t *testing.T) {
  1377. session := createSession(t)
  1378. defer session.Close()
  1379. keyspaceMetadata, err := getKeyspaceMetadata(session, "gocql_test")
  1380. if err != nil {
  1381. t.Fatalf("failed to query the keyspace metadata with err: %v", err)
  1382. }
  1383. if keyspaceMetadata == nil {
  1384. t.Fatal("failed to query the keyspace metadata, nil returned")
  1385. }
  1386. if keyspaceMetadata.Name != "gocql_test" {
  1387. t.Errorf("Expected keyspace name to be 'gocql' but was '%s'", keyspaceMetadata.Name)
  1388. }
  1389. if keyspaceMetadata.StrategyClass != "org.apache.cassandra.locator.SimpleStrategy" {
  1390. t.Errorf("Expected replication strategy class to be 'org.apache.cassandra.locator.SimpleStrategy' but was '%s'", keyspaceMetadata.StrategyClass)
  1391. }
  1392. if keyspaceMetadata.StrategyOptions == nil {
  1393. t.Error("Expected replication strategy options map but was nil")
  1394. }
  1395. rfStr, ok := keyspaceMetadata.StrategyOptions["replication_factor"]
  1396. if !ok {
  1397. t.Fatalf("Expected strategy option 'replication_factor' but was not found in %v", keyspaceMetadata.StrategyOptions)
  1398. }
  1399. rfInt, err := strconv.Atoi(rfStr.(string))
  1400. if err != nil {
  1401. t.Fatalf("Error converting string to int with err: %v", err)
  1402. }
  1403. if rfInt != *flagRF {
  1404. t.Errorf("Expected replication factor to be %d but was %d", *flagRF, rfInt)
  1405. }
  1406. }
  1407. // Integration test of just querying for data from the system.schema_keyspace table where the keyspace DOES NOT exist.
  1408. func TestGetKeyspaceMetadataFails(t *testing.T) {
  1409. session := createSession(t)
  1410. defer session.Close()
  1411. _, err := getKeyspaceMetadata(session, "gocql_keyspace_does_not_exist")
  1412. if err != ErrKeyspaceDoesNotExist || err == nil {
  1413. t.Fatalf("Expected error of type ErrKeySpaceDoesNotExist. Instead, error was %v", err)
  1414. }
  1415. }
  1416. // Integration test of just querying for data from the system.schema_columnfamilies table
  1417. func TestGetTableMetadata(t *testing.T) {
  1418. session := createSession(t)
  1419. defer session.Close()
  1420. if err := createTable(session, "CREATE TABLE gocql_test.test_table_metadata (first_id int, second_id int, third_id int, PRIMARY KEY (first_id, second_id))"); err != nil {
  1421. t.Fatalf("failed to create table with error '%v'", err)
  1422. }
  1423. tables, err := getTableMetadata(session, "gocql_test")
  1424. if err != nil {
  1425. t.Fatalf("failed to query the table metadata with err: %v", err)
  1426. }
  1427. if tables == nil {
  1428. t.Fatal("failed to query the table metadata, nil returned")
  1429. }
  1430. var testTable *TableMetadata
  1431. // verify all tables have minimum expected data
  1432. for i := range tables {
  1433. table := &tables[i]
  1434. if table.Name == "" {
  1435. t.Errorf("Expected table name to be set, but it was empty: index=%d metadata=%+v", i, table)
  1436. }
  1437. if table.Keyspace != "gocql_test" {
  1438. t.Errorf("Expected keyspace for '%s' table metadata to be 'gocql_test' but was '%s'", table.Name, table.Keyspace)
  1439. }
  1440. if *flagProto < 4 {
  1441. // TODO(zariel): there has to be a better way to detect what metadata version
  1442. // we are in, and a better way to structure the code so that it is abstracted away
  1443. // from us here
  1444. if table.KeyValidator == "" {
  1445. t.Errorf("Expected key validator to be set for table %s", table.Name)
  1446. }
  1447. if table.Comparator == "" {
  1448. t.Errorf("Expected comparator to be set for table %s", table.Name)
  1449. }
  1450. if table.DefaultValidator == "" {
  1451. t.Errorf("Expected default validator to be set for table %s", table.Name)
  1452. }
  1453. }
  1454. // these fields are not set until the metadata is compiled
  1455. if table.PartitionKey != nil {
  1456. t.Errorf("Did not expect partition key for table %s", table.Name)
  1457. }
  1458. if table.ClusteringColumns != nil {
  1459. t.Errorf("Did not expect clustering columns for table %s", table.Name)
  1460. }
  1461. if table.Columns != nil {
  1462. t.Errorf("Did not expect columns for table %s", table.Name)
  1463. }
  1464. // for the next part of the test after this loop, find the metadata for the test table
  1465. if table.Name == "test_table_metadata" {
  1466. testTable = table
  1467. }
  1468. }
  1469. // verify actual values on the test tables
  1470. if testTable == nil {
  1471. t.Fatal("Expected table metadata for name 'test_table_metadata'")
  1472. }
  1473. if *flagProto == protoVersion1 {
  1474. if testTable.KeyValidator != "org.apache.cassandra.db.marshal.Int32Type" {
  1475. t.Errorf("Expected test_table_metadata key validator to be 'org.apache.cassandra.db.marshal.Int32Type' but was '%s'", testTable.KeyValidator)
  1476. }
  1477. if testTable.Comparator != "org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.UTF8Type)" {
  1478. t.Errorf("Expected test_table_metadata key validator to be 'org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.UTF8Type)' but was '%s'", testTable.Comparator)
  1479. }
  1480. if testTable.DefaultValidator != "org.apache.cassandra.db.marshal.BytesType" {
  1481. t.Errorf("Expected test_table_metadata key validator to be 'org.apache.cassandra.db.marshal.BytesType' but was '%s'", testTable.DefaultValidator)
  1482. }
  1483. expectedKeyAliases := []string{"first_id"}
  1484. if !reflect.DeepEqual(testTable.KeyAliases, expectedKeyAliases) {
  1485. t.Errorf("Expected key aliases %v but was %v", expectedKeyAliases, testTable.KeyAliases)
  1486. }
  1487. expectedColumnAliases := []string{"second_id"}
  1488. if !reflect.DeepEqual(testTable.ColumnAliases, expectedColumnAliases) {
  1489. t.Errorf("Expected key aliases %v but was %v", expectedColumnAliases, testTable.ColumnAliases)
  1490. }
  1491. }
  1492. if testTable.ValueAlias != "" {
  1493. t.Errorf("Expected value alias '' but was '%s'", testTable.ValueAlias)
  1494. }
  1495. }
  1496. // Integration test of just querying for data from the system.schema_columns table
  1497. func TestGetColumnMetadata(t *testing.T) {
  1498. session := createSession(t)
  1499. defer session.Close()
  1500. if err := createTable(session, "CREATE TABLE gocql_test.test_column_metadata (first_id int, second_id int, third_id int, PRIMARY KEY (first_id, second_id))"); err != nil {
  1501. t.Fatalf("failed to create table with error '%v'", err)
  1502. }
  1503. if err := session.Query("CREATE INDEX index_column_metadata ON test_column_metadata ( third_id )").Exec(); err != nil {
  1504. t.Fatalf("failed to create index with err: %v", err)
  1505. }
  1506. columns, err := getColumnMetadata(session, "gocql_test")
  1507. if err != nil {
  1508. t.Fatalf("failed to query column metadata with err: %v", err)
  1509. }
  1510. if columns == nil {
  1511. t.Fatal("failed to query column metadata, nil returned")
  1512. }
  1513. testColumns := map[string]*ColumnMetadata{}
  1514. // verify actual values on the test columns
  1515. for i := range columns {
  1516. column := &columns[i]
  1517. if column.Name == "" {
  1518. t.Errorf("Expected column name to be set, but it was empty: index=%d metadata=%+v", i, column)
  1519. }
  1520. if column.Table == "" {
  1521. t.Errorf("Expected column %s table name to be set, but it was empty", column.Name)
  1522. }
  1523. if column.Keyspace != "gocql_test" {
  1524. t.Errorf("Expected column %s keyspace name to be 'gocql_test', but it was '%s'", column.Name, column.Keyspace)
  1525. }
  1526. if column.Kind == ColumnUnkownKind {
  1527. t.Errorf("Expected column %s kind to be set, but it was empty", column.Name)
  1528. }
  1529. if session.cfg.ProtoVersion == 1 && column.Kind != ColumnRegular {
  1530. t.Errorf("Expected column %s kind to be set to 'regular' for proto V1 but it was '%s'", column.Name, column.Kind)
  1531. }
  1532. if column.Validator == "" {
  1533. t.Errorf("Expected column %s validator to be set, but it was empty", column.Name)
  1534. }
  1535. // find the test table columns for the next step after this loop
  1536. if column.Table == "test_column_metadata" {
  1537. testColumns[column.Name] = column
  1538. }
  1539. }
  1540. if *flagProto == 1 {
  1541. // V1 proto only returns "regular columns"
  1542. if len(testColumns) != 1 {
  1543. t.Errorf("Expected 1 test columns but there were %d", len(testColumns))
  1544. }
  1545. thirdID, found := testColumns["third_id"]
  1546. if !found {
  1547. t.Fatalf("Expected to find column 'third_id' metadata but there was only %v", testColumns)
  1548. }
  1549. if thirdID.Kind != ColumnRegular {
  1550. t.Errorf("Expected %s column kind to be '%s' but it was '%s'", thirdID.Name, ColumnRegular, thirdID.Kind)
  1551. }
  1552. if thirdID.Index.Name != "index_column_metadata" {
  1553. t.Errorf("Expected %s column index name to be 'index_column_metadata' but it was '%s'", thirdID.Name, thirdID.Index.Name)
  1554. }
  1555. } else {
  1556. if len(testColumns) != 3 {
  1557. t.Errorf("Expected 3 test columns but there were %d", len(testColumns))
  1558. }
  1559. firstID, found := testColumns["first_id"]
  1560. if !found {
  1561. t.Fatalf("Expected to find column 'first_id' metadata but there was only %v", testColumns)
  1562. }
  1563. secondID, found := testColumns["second_id"]
  1564. if !found {
  1565. t.Fatalf("Expected to find column 'second_id' metadata but there was only %v", testColumns)
  1566. }
  1567. thirdID, found := testColumns["third_id"]
  1568. if !found {
  1569. t.Fatalf("Expected to find column 'third_id' metadata but there was only %v", testColumns)
  1570. }
  1571. if firstID.Kind != ColumnPartitionKey {
  1572. t.Errorf("Expected %s column kind to be '%s' but it was '%s'", firstID.Name, ColumnPartitionKey, firstID.Kind)
  1573. }
  1574. if secondID.Kind != ColumnClusteringKey {
  1575. t.Errorf("Expected %s column kind to be '%s' but it was '%s'", secondID.Name, ColumnClusteringKey, secondID.Kind)
  1576. }
  1577. if thirdID.Kind != ColumnRegular {
  1578. t.Errorf("Expected %s column kind to be '%s' but it was '%s'", thirdID.Name, ColumnRegular, thirdID.Kind)
  1579. }
  1580. if !session.useSystemSchema && thirdID.Index.Name != "index_column_metadata" {
  1581. // TODO(zariel): update metadata to scan index from system_schema
  1582. t.Errorf("Expected %s column index name to be 'index_column_metadata' but it was '%s'", thirdID.Name, thirdID.Index.Name)
  1583. }
  1584. }
  1585. }
  1586. // Integration test of querying and composition the keyspace metadata
  1587. func TestKeyspaceMetadata(t *testing.T) {
  1588. session := createSession(t)
  1589. defer session.Close()
  1590. if err := createTable(session, "CREATE TABLE gocql_test.test_metadata (first_id int, second_id int, third_id int, PRIMARY KEY (first_id, second_id))"); err != nil {
  1591. t.Fatalf("failed to create table with error '%v'", err)
  1592. }
  1593. if err := session.Query("CREATE INDEX index_metadata ON test_metadata ( third_id )").Exec(); err != nil {
  1594. t.Fatalf("failed to create index with err: %v", err)
  1595. }
  1596. keyspaceMetadata, err := session.KeyspaceMetadata("gocql_test")
  1597. if err != nil {
  1598. t.Fatalf("failed to query keyspace metadata with err: %v", err)
  1599. }
  1600. if keyspaceMetadata == nil {
  1601. t.Fatal("expected the keyspace metadata to not be nil, but it was nil")
  1602. }
  1603. if keyspaceMetadata.Name != session.cfg.Keyspace {
  1604. t.Fatalf("Expected the keyspace name to be %s but was %s", session.cfg.Keyspace, keyspaceMetadata.Name)
  1605. }
  1606. if len(keyspaceMetadata.Tables) == 0 {
  1607. t.Errorf("Expected tables but there were none")
  1608. }
  1609. tableMetadata, found := keyspaceMetadata.Tables["test_metadata"]
  1610. if !found {
  1611. t.Fatalf("failed to find the test_metadata table metadata")
  1612. }
  1613. if len(tableMetadata.PartitionKey) != 1 {
  1614. t.Errorf("expected partition key length of 1, but was %d", len(tableMetadata.PartitionKey))
  1615. }
  1616. for i, column := range tableMetadata.PartitionKey {
  1617. if column == nil {
  1618. t.Errorf("partition key column metadata at index %d was nil", i)
  1619. }
  1620. }
  1621. if tableMetadata.PartitionKey[0].Name != "first_id" {
  1622. t.Errorf("Expected the first partition key column to be 'first_id' but was '%s'", tableMetadata.PartitionKey[0].Name)
  1623. }
  1624. if len(tableMetadata.ClusteringColumns) != 1 {
  1625. t.Fatalf("expected clustering columns length of 1, but was %d", len(tableMetadata.ClusteringColumns))
  1626. }
  1627. for i, column := range tableMetadata.ClusteringColumns {
  1628. if column == nil {
  1629. t.Fatalf("clustering column metadata at index %d was nil", i)
  1630. }
  1631. }
  1632. if tableMetadata.ClusteringColumns[0].Name != "second_id" {
  1633. t.Errorf("Expected the first clustering column to be 'second_id' but was '%s'", tableMetadata.ClusteringColumns[0].Name)
  1634. }
  1635. thirdColumn, found := tableMetadata.Columns["third_id"]
  1636. if !found {
  1637. t.Fatalf("Expected a column definition for 'third_id'")
  1638. }
  1639. if !session.useSystemSchema && thirdColumn.Index.Name != "index_metadata" {
  1640. // TODO(zariel): scan index info from system_schema
  1641. t.Errorf("Expected column index named 'index_metadata' but was '%s'", thirdColumn.Index.Name)
  1642. }
  1643. }
  1644. // Integration test of the routing key calculation
  1645. func TestRoutingKey(t *testing.T) {
  1646. session := createSession(t)
  1647. defer session.Close()
  1648. if err := createTable(session, "CREATE TABLE gocql_test.test_single_routing_key (first_id int, second_id int, PRIMARY KEY (first_id, second_id))"); err != nil {
  1649. t.Fatalf("failed to create table with error '%v'", err)
  1650. }
  1651. if err := createTable(session, "CREATE TABLE gocql_test.test_composite_routing_key (first_id int, second_id int, PRIMARY KEY ((first_id, second_id)))"); err != nil {
  1652. t.Fatalf("failed to create table with error '%v'", err)
  1653. }
  1654. routingKeyInfo, err := session.routingKeyInfo(context.Background(), "SELECT * FROM test_single_routing_key WHERE second_id=? AND first_id=?")
  1655. if err != nil {
  1656. t.Fatalf("failed to get routing key info due to error: %v", err)
  1657. }
  1658. if routingKeyInfo == nil {
  1659. t.Fatal("Expected routing key info, but was nil")
  1660. }
  1661. if len(routingKeyInfo.indexes) != 1 {
  1662. t.Fatalf("Expected routing key indexes length to be 1 but was %d", len(routingKeyInfo.indexes))
  1663. }
  1664. if routingKeyInfo.indexes[0] != 1 {
  1665. t.Errorf("Expected routing key index[0] to be 1 but was %d", routingKeyInfo.indexes[0])
  1666. }
  1667. if len(routingKeyInfo.types) != 1 {
  1668. t.Fatalf("Expected routing key types length to be 1 but was %d", len(routingKeyInfo.types))
  1669. }
  1670. if routingKeyInfo.types[0] == nil {
  1671. t.Fatal("Expected routing key types[0] to be non-nil")
  1672. }
  1673. if routingKeyInfo.types[0].Type() != TypeInt {
  1674. t.Fatalf("Expected routing key types[0].Type to be %v but was %v", TypeInt, routingKeyInfo.types[0].Type())
  1675. }
  1676. // verify the cache is working
  1677. routingKeyInfo, err = session.routingKeyInfo(context.Background(), "SELECT * FROM test_single_routing_key WHERE second_id=? AND first_id=?")
  1678. if err != nil {
  1679. t.Fatalf("failed to get routing key info due to error: %v", err)
  1680. }
  1681. if len(routingKeyInfo.indexes) != 1 {
  1682. t.Fatalf("Expected routing key indexes length to be 1 but was %d", len(routingKeyInfo.indexes))
  1683. }
  1684. if routingKeyInfo.indexes[0] != 1 {
  1685. t.Errorf("Expected routing key index[0] to be 1 but was %d", routingKeyInfo.indexes[0])
  1686. }
  1687. if len(routingKeyInfo.types) != 1 {
  1688. t.Fatalf("Expected routing key types length to be 1 but was %d", len(routingKeyInfo.types))
  1689. }
  1690. if routingKeyInfo.types[0] == nil {
  1691. t.Fatal("Expected routing key types[0] to be non-nil")
  1692. }
  1693. if routingKeyInfo.types[0].Type() != TypeInt {
  1694. t.Fatalf("Expected routing key types[0] to be %v but was %v", TypeInt, routingKeyInfo.types[0].Type())
  1695. }
  1696. cacheSize := session.routingKeyInfoCache.lru.Len()
  1697. if cacheSize != 1 {
  1698. t.Errorf("Expected cache size to be 1 but was %d", cacheSize)
  1699. }
  1700. query := session.Query("SELECT * FROM test_single_routing_key WHERE second_id=? AND first_id=?", 1, 2)
  1701. routingKey, err := query.GetRoutingKey()
  1702. if err != nil {
  1703. t.Fatalf("Failed to get routing key due to error: %v", err)
  1704. }
  1705. expectedRoutingKey := []byte{0, 0, 0, 2}
  1706. if !reflect.DeepEqual(expectedRoutingKey, routingKey) {
  1707. t.Errorf("Expected routing key %v but was %v", expectedRoutingKey, routingKey)
  1708. }
  1709. routingKeyInfo, err = session.routingKeyInfo(context.Background(), "SELECT * FROM test_composite_routing_key WHERE second_id=? AND first_id=?")
  1710. if err != nil {
  1711. t.Fatalf("failed to get routing key info due to error: %v", err)
  1712. }
  1713. if routingKeyInfo == nil {
  1714. t.Fatal("Expected routing key info, but was nil")
  1715. }
  1716. if len(routingKeyInfo.indexes) != 2 {
  1717. t.Fatalf("Expected routing key indexes length to be 2 but was %d", len(routingKeyInfo.indexes))
  1718. }
  1719. if routingKeyInfo.indexes[0] != 1 {
  1720. t.Errorf("Expected routing key index[0] to be 1 but was %d", routingKeyInfo.indexes[0])
  1721. }
  1722. if routingKeyInfo.indexes[1] != 0 {
  1723. t.Errorf("Expected routing key index[1] to be 0 but was %d", routingKeyInfo.indexes[1])
  1724. }
  1725. if len(routingKeyInfo.types) != 2 {
  1726. t.Fatalf("Expected routing key types length to be 1 but was %d", len(routingKeyInfo.types))
  1727. }
  1728. if routingKeyInfo.types[0] == nil {
  1729. t.Fatal("Expected routing key types[0] to be non-nil")
  1730. }
  1731. if routingKeyInfo.types[0].Type() != TypeInt {
  1732. t.Fatalf("Expected routing key types[0] to be %v but was %v", TypeInt, routingKeyInfo.types[0].Type())
  1733. }
  1734. if routingKeyInfo.types[1] == nil {
  1735. t.Fatal("Expected routing key types[1] to be non-nil")
  1736. }
  1737. if routingKeyInfo.types[1].Type() != TypeInt {
  1738. t.Fatalf("Expected routing key types[0] to be %v but was %v", TypeInt, routingKeyInfo.types[1].Type())
  1739. }
  1740. query = session.Query("SELECT * FROM test_composite_routing_key WHERE second_id=? AND first_id=?", 1, 2)
  1741. routingKey, err = query.GetRoutingKey()
  1742. if err != nil {
  1743. t.Fatalf("Failed to get routing key due to error: %v", err)
  1744. }
  1745. expectedRoutingKey = []byte{0, 4, 0, 0, 0, 2, 0, 0, 4, 0, 0, 0, 1, 0}
  1746. if !reflect.DeepEqual(expectedRoutingKey, routingKey) {
  1747. t.Errorf("Expected routing key %v but was %v", expectedRoutingKey, routingKey)
  1748. }
  1749. // verify the cache is working
  1750. cacheSize = session.routingKeyInfoCache.lru.Len()
  1751. if cacheSize != 2 {
  1752. t.Errorf("Expected cache size to be 2 but was %d", cacheSize)
  1753. }
  1754. }
  1755. // Integration test of the token-aware policy-based connection pool
  1756. func TestTokenAwareConnPool(t *testing.T) {
  1757. cluster := createCluster()
  1758. cluster.PoolConfig.HostSelectionPolicy = TokenAwareHostPolicy(RoundRobinHostPolicy())
  1759. // force metadata query to page
  1760. cluster.PageSize = 1
  1761. session := createSessionFromCluster(cluster, t)
  1762. defer session.Close()
  1763. expectedPoolSize := cluster.NumConns * len(session.ring.allHosts())
  1764. // wait for pool to fill
  1765. for i := 0; i < 10; i++ {
  1766. if session.pool.Size() == expectedPoolSize {
  1767. break
  1768. }
  1769. time.Sleep(100 * time.Millisecond)
  1770. }
  1771. if expectedPoolSize != session.pool.Size() {
  1772. t.Errorf("Expected pool size %d but was %d", expectedPoolSize, session.pool.Size())
  1773. }
  1774. // add another cf so there are two pages when fetching table metadata from our keyspace
  1775. if err := createTable(session, "CREATE TABLE gocql_test.test_token_aware_other_cf (id int, data text, PRIMARY KEY (id))"); err != nil {
  1776. t.Fatalf("failed to create test_token_aware table with err: %v", err)
  1777. }
  1778. if err := createTable(session, "CREATE TABLE gocql_test.test_token_aware (id int, data text, PRIMARY KEY (id))"); err != nil {
  1779. t.Fatalf("failed to create test_token_aware table with err: %v", err)
  1780. }
  1781. query := session.Query("INSERT INTO test_token_aware (id, data) VALUES (?,?)", 42, "8 * 6 =")
  1782. if err := query.Exec(); err != nil {
  1783. t.Fatalf("failed to insert with err: %v", err)
  1784. }
  1785. query = session.Query("SELECT data FROM test_token_aware where id = ?", 42).Consistency(One)
  1786. var data string
  1787. if err := query.Scan(&data); err != nil {
  1788. t.Error(err)
  1789. }
  1790. // TODO add verification that the query went to the correct host
  1791. }
  1792. func TestNegativeStream(t *testing.T) {
  1793. session := createSession(t)
  1794. defer session.Close()
  1795. conn := getRandomConn(t, session)
  1796. const stream = -50
  1797. writer := frameWriterFunc(func(f *framer, streamID int) error {
  1798. f.writeHeader(0, opOptions, stream)
  1799. return f.finishWrite()
  1800. })
  1801. frame, err := conn.exec(context.Background(), writer, nil)
  1802. if err == nil {
  1803. t.Fatalf("expected to get an error on stream %d", stream)
  1804. } else if frame != nil {
  1805. t.Fatalf("expected to get nil frame got %+v", frame)
  1806. }
  1807. }
  1808. func TestManualQueryPaging(t *testing.T) {
  1809. const rowsToInsert = 5
  1810. session := createSession(t)
  1811. defer session.Close()
  1812. if err := createTable(session, "CREATE TABLE gocql_test.testManualPaging (id int, count int, PRIMARY KEY (id))"); err != nil {
  1813. t.Fatal(err)
  1814. }
  1815. for i := 0; i < rowsToInsert; i++ {
  1816. err := session.Query("INSERT INTO testManualPaging(id, count) VALUES(?, ?)", i, i*i).Exec()
  1817. if err != nil {
  1818. t.Fatal(err)
  1819. }
  1820. }
  1821. // disable auto paging, 1 page per iteration
  1822. query := session.Query("SELECT id, count FROM testManualPaging").PageState(nil).PageSize(2)
  1823. var id, count, fetched int
  1824. iter := query.Iter()
  1825. // NOTE: this isnt very indicative of how it should be used, the idea is that
  1826. // the page state is returned to some client who will send it back to manually
  1827. // page through the results.
  1828. for {
  1829. for iter.Scan(&id, &count) {
  1830. if count != (id * id) {
  1831. t.Fatalf("got wrong value from iteration: got %d expected %d", count, id*id)
  1832. }
  1833. fetched++
  1834. }
  1835. if len(iter.PageState()) > 0 {
  1836. // more pages
  1837. iter = query.PageState(iter.PageState()).Iter()
  1838. } else {
  1839. break
  1840. }
  1841. }
  1842. if err := iter.Close(); err != nil {
  1843. t.Fatal(err)
  1844. }
  1845. if fetched != rowsToInsert {
  1846. t.Fatalf("expected to fetch %d rows got %d", rowsToInsert, fetched)
  1847. }
  1848. }
  1849. func TestLexicalUUIDType(t *testing.T) {
  1850. session := createSession(t)
  1851. defer session.Close()
  1852. if err := createTable(session, `CREATE TABLE gocql_test.test_lexical_uuid (
  1853. key varchar,
  1854. column1 'org.apache.cassandra.db.marshal.LexicalUUIDType',
  1855. value int,
  1856. PRIMARY KEY (key, column1)
  1857. )`); err != nil {
  1858. t.Fatal("create:", err)
  1859. }
  1860. key := TimeUUID().String()
  1861. column1 := TimeUUID()
  1862. err := session.Query("INSERT INTO test_lexical_uuid(key, column1, value) VALUES(?, ?, ?)", key, column1, 55).Exec()
  1863. if err != nil {
  1864. t.Fatal(err)
  1865. }
  1866. var gotUUID UUID
  1867. if err := session.Query("SELECT column1 from test_lexical_uuid where key = ? AND column1 = ?", key, column1).Scan(&gotUUID); err != nil {
  1868. t.Fatal(err)
  1869. }
  1870. if gotUUID != column1 {
  1871. t.Errorf("got %s, expected %s", gotUUID, column1)
  1872. }
  1873. }
  1874. // Issue 475
  1875. func TestSessionBindRoutingKey(t *testing.T) {
  1876. cluster := createCluster()
  1877. cluster.PoolConfig.HostSelectionPolicy = TokenAwareHostPolicy(RoundRobinHostPolicy())
  1878. session := createSessionFromCluster(cluster, t)
  1879. defer session.Close()
  1880. if err := createTable(session, `CREATE TABLE gocql_test.test_bind_routing_key (
  1881. key varchar,
  1882. value int,
  1883. PRIMARY KEY (key)
  1884. )`); err != nil {
  1885. t.Fatal(err)
  1886. }
  1887. const (
  1888. key = "routing-key"
  1889. value = 5
  1890. )
  1891. fn := func(info *QueryInfo) ([]interface{}, error) {
  1892. return []interface{}{key, value}, nil
  1893. }
  1894. q := session.Bind("INSERT INTO test_bind_routing_key(key, value) VALUES(?, ?)", fn)
  1895. if err := q.Exec(); err != nil {
  1896. t.Fatal(err)
  1897. }
  1898. }
  1899. func TestJSONSupport(t *testing.T) {
  1900. if *flagProto < 4 {
  1901. t.Skip("skipping JSON support on proto < 4")
  1902. }
  1903. session := createSession(t)
  1904. defer session.Close()
  1905. if err := createTable(session, `CREATE TABLE gocql_test.test_json (
  1906. id text PRIMARY KEY,
  1907. age int,
  1908. state text
  1909. )`); err != nil {
  1910. t.Fatal(err)
  1911. }
  1912. err := session.Query("INSERT INTO test_json JSON ?", `{"id": "user123", "age": 42, "state": "TX"}`).Exec()
  1913. if err != nil {
  1914. t.Fatal(err)
  1915. }
  1916. var (
  1917. id string
  1918. age int
  1919. state string
  1920. )
  1921. err = session.Query("SELECT id, age, state FROM test_json WHERE id = ?", "user123").Scan(&id, &age, &state)
  1922. if err != nil {
  1923. t.Fatal(err)
  1924. }
  1925. if id != "user123" {
  1926. t.Errorf("got id %q expected %q", id, "user123")
  1927. }
  1928. if age != 42 {
  1929. t.Errorf("got age %d expected %d", age, 42)
  1930. }
  1931. if state != "TX" {
  1932. t.Errorf("got state %q expected %q", state, "TX")
  1933. }
  1934. }
  1935. func TestUDF(t *testing.T) {
  1936. if *flagProto < 4 {
  1937. t.Skip("skipping UDF support on proto < 4")
  1938. }
  1939. session := createSession(t)
  1940. defer session.Close()
  1941. const query = `CREATE OR REPLACE FUNCTION uniq(state set<text>, val text)
  1942. CALLED ON NULL INPUT RETURNS set<text> LANGUAGE java
  1943. AS 'state.add(val); return state;'`
  1944. err := session.Query(query).Exec()
  1945. if err != nil {
  1946. t.Fatal(err)
  1947. }
  1948. }
  1949. func TestDiscoverViaProxy(t *testing.T) {
  1950. // This (complicated) test tests that when the driver is given an initial host
  1951. // that is infact a proxy it discovers the rest of the ring behind the proxy
  1952. // and does not store the proxies address as a host in its connection pool.
  1953. // See https://github.com/gocql/gocql/issues/481
  1954. proxy, err := net.Listen("tcp", "localhost:0")
  1955. if err != nil {
  1956. t.Fatalf("unable to create proxy listener: %v", err)
  1957. }
  1958. var (
  1959. wg sync.WaitGroup
  1960. mu sync.Mutex
  1961. proxyConns []net.Conn
  1962. closed bool
  1963. )
  1964. go func(wg *sync.WaitGroup) {
  1965. cassandraAddr := JoinHostPort(clusterHosts[0], 9042)
  1966. cassandra := func() (net.Conn, error) {
  1967. return net.Dial("tcp", cassandraAddr)
  1968. }
  1969. proxyFn := func(wg *sync.WaitGroup, from, to net.Conn) {
  1970. defer wg.Done()
  1971. _, err := io.Copy(to, from)
  1972. if err != nil {
  1973. mu.Lock()
  1974. if !closed {
  1975. t.Error(err)
  1976. }
  1977. mu.Unlock()
  1978. }
  1979. }
  1980. // handle dials cassandra and then proxies requests and reponsess. It waits
  1981. // for both the read and write side of the TCP connection to close before
  1982. // returning.
  1983. handle := func(conn net.Conn) error {
  1984. defer conn.Close()
  1985. cass, err := cassandra()
  1986. if err != nil {
  1987. return err
  1988. }
  1989. mu.Lock()
  1990. proxyConns = append(proxyConns, cass)
  1991. mu.Unlock()
  1992. defer cass.Close()
  1993. var wg sync.WaitGroup
  1994. wg.Add(1)
  1995. go proxyFn(&wg, conn, cass)
  1996. wg.Add(1)
  1997. go proxyFn(&wg, cass, conn)
  1998. wg.Wait()
  1999. return nil
  2000. }
  2001. for {
  2002. // proxy just accepts connections and then proxies them to cassandra,
  2003. // it runs until it is closed.
  2004. conn, err := proxy.Accept()
  2005. if err != nil {
  2006. mu.Lock()
  2007. if !closed {
  2008. t.Error(err)
  2009. }
  2010. mu.Unlock()
  2011. return
  2012. }
  2013. mu.Lock()
  2014. proxyConns = append(proxyConns, conn)
  2015. mu.Unlock()
  2016. wg.Add(1)
  2017. go func(conn net.Conn) {
  2018. defer wg.Done()
  2019. if err := handle(conn); err != nil {
  2020. t.Error(err)
  2021. return
  2022. }
  2023. }(conn)
  2024. }
  2025. }(&wg)
  2026. defer wg.Wait()
  2027. proxyAddr := proxy.Addr().String()
  2028. cluster := createCluster()
  2029. cluster.NumConns = 1
  2030. // initial host is the proxy address
  2031. cluster.Hosts = []string{proxyAddr}
  2032. session := createSessionFromCluster(cluster, t)
  2033. defer session.Close()
  2034. if !session.hostSource.localHasRpcAddr {
  2035. t.Skip("Target cluster does not have rpc_address in system.local.")
  2036. goto close
  2037. }
  2038. // we shouldnt need this but to be safe
  2039. time.Sleep(1 * time.Second)
  2040. session.pool.mu.RLock()
  2041. for _, host := range clusterHosts {
  2042. if _, ok := session.pool.hostConnPools[host]; !ok {
  2043. t.Errorf("missing host in pool after discovery: %q", host)
  2044. }
  2045. }
  2046. session.pool.mu.RUnlock()
  2047. close:
  2048. mu.Lock()
  2049. closed = true
  2050. if err := proxy.Close(); err != nil {
  2051. t.Log(err)
  2052. }
  2053. for _, conn := range proxyConns {
  2054. if err := conn.Close(); err != nil {
  2055. t.Log(err)
  2056. }
  2057. }
  2058. mu.Unlock()
  2059. }
  2060. func TestUnmarshallNestedTypes(t *testing.T) {
  2061. if *flagProto < protoVersion3 {
  2062. t.Skip("can not have frozen types in cassandra < 2.1.3")
  2063. }
  2064. session := createSession(t)
  2065. defer session.Close()
  2066. if err := createTable(session, `CREATE TABLE gocql_test.test_557 (
  2067. id text PRIMARY KEY,
  2068. val list<frozen<map<text, text> > >
  2069. )`); err != nil {
  2070. t.Fatal(err)
  2071. }
  2072. m := []map[string]string{
  2073. {"key1": "val1"},
  2074. {"key2": "val2"},
  2075. }
  2076. const id = "key"
  2077. err := session.Query("INSERT INTO test_557(id, val) VALUES(?, ?)", id, m).Exec()
  2078. if err != nil {
  2079. t.Fatal(err)
  2080. }
  2081. var data []map[string]string
  2082. if err := session.Query("SELECT val FROM test_557 WHERE id = ?", id).Scan(&data); err != nil {
  2083. t.Fatal(err)
  2084. }
  2085. if !reflect.DeepEqual(data, m) {
  2086. t.Fatalf("%+#v != %+#v", data, m)
  2087. }
  2088. }
  2089. func TestSchemaReset(t *testing.T) {
  2090. if flagCassVersion.Major == 0 || (flagCassVersion.Before(2, 1, 3)) {
  2091. t.Skipf("skipping TestSchemaReset due to CASSANDRA-7910 in Cassandra <2.1.3 version=%v", flagCassVersion)
  2092. }
  2093. cluster := createCluster()
  2094. cluster.NumConns = 1
  2095. session := createSessionFromCluster(cluster, t)
  2096. defer session.Close()
  2097. if err := createTable(session, `CREATE TABLE gocql_test.test_schema_reset (
  2098. id text PRIMARY KEY)`); err != nil {
  2099. t.Fatal(err)
  2100. }
  2101. const key = "test"
  2102. err := session.Query("INSERT INTO test_schema_reset(id) VALUES(?)", key).Exec()
  2103. if err != nil {
  2104. t.Fatal(err)
  2105. }
  2106. var id string
  2107. err = session.Query("SELECT * FROM test_schema_reset WHERE id=?", key).Scan(&id)
  2108. if err != nil {
  2109. t.Fatal(err)
  2110. } else if id != key {
  2111. t.Fatalf("expected to get id=%q got=%q", key, id)
  2112. }
  2113. if err := createTable(session, `ALTER TABLE gocql_test.test_schema_reset ADD val text`); err != nil {
  2114. t.Fatal(err)
  2115. }
  2116. const expVal = "test-val"
  2117. err = session.Query("INSERT INTO test_schema_reset(id, val) VALUES(?, ?)", key, expVal).Exec()
  2118. if err != nil {
  2119. t.Fatal(err)
  2120. }
  2121. var val string
  2122. err = session.Query("SELECT * FROM test_schema_reset WHERE id=?", key).Scan(&id, &val)
  2123. if err != nil {
  2124. t.Fatal(err)
  2125. }
  2126. if id != key {
  2127. t.Errorf("expected to get id=%q got=%q", key, id)
  2128. }
  2129. if val != expVal {
  2130. t.Errorf("expected to get val=%q got=%q", expVal, val)
  2131. }
  2132. }
  2133. func TestCreateSession_DontSwallowError(t *testing.T) {
  2134. t.Skip("This test is bad, and the resultant error from cassandra changes between versions")
  2135. cluster := createCluster()
  2136. cluster.ProtoVersion = 0x100
  2137. session, err := cluster.CreateSession()
  2138. if err == nil {
  2139. session.Close()
  2140. t.Fatal("expected to get an error for unsupported protocol")
  2141. }
  2142. if flagCassVersion.Major < 3 {
  2143. // TODO: we should get a distinct error type here which include the underlying
  2144. // cassandra error about the protocol version, for now check this here.
  2145. if !strings.Contains(err.Error(), "Invalid or unsupported protocol version") {
  2146. t.Fatalf(`expcted to get error "unsupported protocol version" got: %q`, err)
  2147. }
  2148. } else {
  2149. if !strings.Contains(err.Error(), "unsupported response version") {
  2150. t.Fatalf(`expcted to get error "unsupported response version" got: %q`, err)
  2151. }
  2152. }
  2153. }
  2154. func TestControl_DiscoverProtocol(t *testing.T) {
  2155. cluster := createCluster()
  2156. cluster.ProtoVersion = 0
  2157. session, err := cluster.CreateSession()
  2158. if err != nil {
  2159. t.Fatal(err)
  2160. }
  2161. defer session.Close()
  2162. if session.cfg.ProtoVersion == 0 {
  2163. t.Fatal("did not discovery protocol")
  2164. }
  2165. }
  2166. // TestUnsetCol verify unset column will not replace an existing column
  2167. func TestUnsetCol(t *testing.T) {
  2168. if *flagProto < 4 {
  2169. t.Skip("Unset Values are not supported in protocol < 4")
  2170. }
  2171. session := createSession(t)
  2172. defer session.Close()
  2173. if err := createTable(session, "CREATE TABLE gocql_test.testUnsetInsert (id int, my_int int, my_text text, PRIMARY KEY (id))"); err != nil {
  2174. t.Fatalf("failed to create table with error '%v'", err)
  2175. }
  2176. if err := session.Query("INSERT INTO testUnSetInsert (id,my_int,my_text) VALUES (?,?,?)", 1, 2, "3").Exec(); err != nil {
  2177. t.Fatalf("failed to insert with err: %v", err)
  2178. }
  2179. if err := session.Query("INSERT INTO testUnSetInsert (id,my_int,my_text) VALUES (?,?,?)", 1, UnsetValue, UnsetValue).Exec(); err != nil {
  2180. t.Fatalf("failed to insert with err: %v", err)
  2181. }
  2182. var id, mInt int
  2183. var mText string
  2184. if err := session.Query("SELECT id, my_int ,my_text FROM testUnsetInsert").Scan(&id, &mInt, &mText); err != nil {
  2185. t.Fatalf("failed to select with err: %v", err)
  2186. } else if id != 1 || mInt != 2 || mText != "3" {
  2187. t.Fatalf("Expected results: 1, 2, \"3\", got %v, %v, %v", id, mInt, mText)
  2188. }
  2189. }
  2190. // TestUnsetColBatch verify unset column will not replace a column in batch
  2191. func TestUnsetColBatch(t *testing.T) {
  2192. if *flagProto < 4 {
  2193. t.Skip("Unset Values are not supported in protocol < 4")
  2194. }
  2195. session := createSession(t)
  2196. defer session.Close()
  2197. if err := createTable(session, "CREATE TABLE gocql_test.batchUnsetInsert (id int, my_int int, my_text text, PRIMARY KEY (id))"); err != nil {
  2198. t.Fatalf("failed to create table with error '%v'", err)
  2199. }
  2200. b := session.NewBatch(LoggedBatch)
  2201. b.Query("INSERT INTO gocql_test.batchUnsetInsert(id, my_int, my_text) VALUES (?,?,?)", 1, 1, UnsetValue)
  2202. b.Query("INSERT INTO gocql_test.batchUnsetInsert(id, my_int, my_text) VALUES (?,?,?)", 1, UnsetValue, "")
  2203. b.Query("INSERT INTO gocql_test.batchUnsetInsert(id, my_int, my_text) VALUES (?,?,?)", 2, 2, UnsetValue)
  2204. if err := session.ExecuteBatch(b); err != nil {
  2205. t.Fatalf("query failed. %v", err)
  2206. } else {
  2207. if b.Attempts() < 1 {
  2208. t.Fatal("expected at least 1 attempt, but got 0")
  2209. }
  2210. if b.Latency() <= 0 {
  2211. t.Fatalf("expected latency to be greater than 0, but got %v instead.", b.Latency())
  2212. }
  2213. }
  2214. var id, mInt, count int
  2215. var mText string
  2216. if err := session.Query("SELECT count(*) FROM gocql_test.batchUnsetInsert;").Scan(&count); err != nil {
  2217. t.Fatalf("Failed to select with err: %v", err)
  2218. } else if count != 2 {
  2219. t.Fatalf("Expected Batch Insert count 2, got %v", count)
  2220. }
  2221. if err := session.Query("SELECT id, my_int ,my_text FROM gocql_test.batchUnsetInsert where id=1;").Scan(&id, &mInt, &mText); err != nil {
  2222. t.Fatalf("failed to select with err: %v", err)
  2223. } else if id != mInt {
  2224. t.Fatalf("expected id, my_int to be 1, got %v and %v", id, mInt)
  2225. }
  2226. }