metadata.go 20 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859
  1. // Copyright (c) 2015 The gocql Authors. All rights reserved.
  2. // Use of this source code is governed by a BSD-style
  3. // license that can be found in the LICENSE file.
  4. package gocql
  5. import (
  6. "encoding/hex"
  7. "encoding/json"
  8. "fmt"
  9. "log"
  10. "strconv"
  11. "strings"
  12. "sync"
  13. )
  14. // schema metadata for a keyspace
  15. type KeyspaceMetadata struct {
  16. Name string
  17. DurableWrites bool
  18. StrategyClass string
  19. StrategyOptions map[string]interface{}
  20. Tables map[string]*TableMetadata
  21. }
  22. // schema metadata for a table (a.k.a. column family)
  23. type TableMetadata struct {
  24. Keyspace string
  25. Name string
  26. KeyValidator string
  27. Comparator string
  28. DefaultValidator string
  29. KeyAliases []string
  30. ColumnAliases []string
  31. ValueAlias string
  32. PartitionKey []*ColumnMetadata
  33. ClusteringColumns []*ColumnMetadata
  34. Columns map[string]*ColumnMetadata
  35. }
  36. // schema metadata for a column
  37. type ColumnMetadata struct {
  38. Keyspace string
  39. Table string
  40. Name string
  41. ComponentIndex int
  42. Kind string
  43. Validator string
  44. Type TypeInfo
  45. Order ColumnOrder
  46. Index ColumnIndexMetadata
  47. }
  48. // the ordering of the column with regard to its comparator
  49. type ColumnOrder bool
  50. const (
  51. ASC ColumnOrder = false
  52. DESC = true
  53. )
  54. type ColumnIndexMetadata struct {
  55. Name string
  56. Type string
  57. Options map[string]interface{}
  58. }
  59. // Column kind values
  60. const (
  61. PARTITION_KEY = "partition_key"
  62. CLUSTERING_KEY = "clustering_key"
  63. REGULAR = "regular"
  64. )
  65. // default alias values
  66. const (
  67. DEFAULT_KEY_ALIAS = "key"
  68. DEFAULT_COLUMN_ALIAS = "column"
  69. DEFAULT_VALUE_ALIAS = "value"
  70. )
  71. // queries the cluster for schema information for a specific keyspace
  72. type schemaDescriber struct {
  73. session *Session
  74. mu sync.Mutex
  75. cache map[string]*KeyspaceMetadata
  76. }
  77. func newSchemaDescriber(session *Session) *schemaDescriber {
  78. return &schemaDescriber{
  79. session: session,
  80. cache: map[string]*KeyspaceMetadata{},
  81. }
  82. }
  83. func (s *schemaDescriber) getSchema(keyspaceName string) (*KeyspaceMetadata, error) {
  84. s.mu.Lock()
  85. defer s.mu.Unlock()
  86. // TODO handle schema change events
  87. metadata, found := s.cache[keyspaceName]
  88. if !found {
  89. // refresh the cache for this keyspace
  90. err := s.refreshSchema(keyspaceName)
  91. if err != nil {
  92. return nil, err
  93. }
  94. metadata = s.cache[keyspaceName]
  95. }
  96. return metadata, nil
  97. }
  98. func (s *schemaDescriber) refreshSchema(keyspaceName string) error {
  99. var err error
  100. // query the system keyspace for schema data
  101. // TODO retrieve concurrently
  102. keyspace, err := getKeyspaceMetadata(s.session, keyspaceName)
  103. if err != nil {
  104. return err
  105. }
  106. tables, err := getTableMetadata(s.session, keyspaceName)
  107. if err != nil {
  108. return err
  109. }
  110. columns, err := getColumnMetadata(s.session, keyspaceName)
  111. if err != nil {
  112. return err
  113. }
  114. // organize the schema data
  115. compileMetadata(s.session.cfg.ProtoVersion, keyspace, tables, columns)
  116. // update the cache
  117. s.cache[keyspaceName] = keyspace
  118. return nil
  119. }
  120. // "compiles" keyspace, table, and column metadata for a keyspace together
  121. // linking the metadata objects together and calculating the partition key
  122. // and clustering key.
  123. func compileMetadata(
  124. protoVersion int,
  125. keyspace *KeyspaceMetadata,
  126. tables []TableMetadata,
  127. columns []ColumnMetadata,
  128. ) {
  129. keyspace.Tables = make(map[string]*TableMetadata)
  130. for i := range tables {
  131. tables[i].Columns = make(map[string]*ColumnMetadata)
  132. keyspace.Tables[tables[i].Name] = &tables[i]
  133. }
  134. // add columns from the schema data
  135. for i := range columns {
  136. // decode the validator for TypeInfo and order
  137. validatorParsed := parseType(columns[i].Validator)
  138. columns[i].Type = validatorParsed.types[0]
  139. columns[i].Order = ASC
  140. if validatorParsed.reversed[0] {
  141. columns[i].Order = DESC
  142. }
  143. table := keyspace.Tables[columns[i].Table]
  144. table.Columns[columns[i].Name] = &columns[i]
  145. }
  146. if protoVersion == 1 {
  147. compileV1Metadata(tables)
  148. } else {
  149. compileV2Metadata(tables)
  150. }
  151. }
  152. // V1 protocol does not return as much column metadata as V2+ so determining
  153. // PartitionKey and ClusterColumns is more complex
  154. func compileV1Metadata(tables []TableMetadata) {
  155. for i := range tables {
  156. table := &tables[i]
  157. // decode the key validator
  158. keyValidatorParsed := parseType(table.KeyValidator)
  159. // decode the comparator
  160. comparatorParsed := parseType(table.Comparator)
  161. // the partition key length is the same as the number of types in the
  162. // key validator
  163. table.PartitionKey = make([]*ColumnMetadata, len(keyValidatorParsed.types))
  164. // V1 protocol only returns "regular" columns from
  165. // system.schema_columns (there is no type field for columns)
  166. // so the alias information is used to
  167. // create the partition key and clustering columns
  168. // construct the partition key from the alias
  169. for i := range table.PartitionKey {
  170. var alias string
  171. if len(table.KeyAliases) > i {
  172. alias = table.KeyAliases[i]
  173. } else if i == 0 {
  174. alias = DEFAULT_KEY_ALIAS
  175. } else {
  176. alias = DEFAULT_KEY_ALIAS + strconv.Itoa(i+1)
  177. }
  178. column := &ColumnMetadata{
  179. Keyspace: table.Keyspace,
  180. Table: table.Name,
  181. Name: alias,
  182. Type: keyValidatorParsed.types[i],
  183. Kind: PARTITION_KEY,
  184. ComponentIndex: i,
  185. }
  186. table.PartitionKey[i] = column
  187. table.Columns[alias] = column
  188. }
  189. // determine the number of clustering columns
  190. size := len(comparatorParsed.types)
  191. if comparatorParsed.isComposite {
  192. if len(comparatorParsed.collections) != 0 ||
  193. (len(table.ColumnAliases) == size-1 &&
  194. comparatorParsed.types[size-1].Type() == TypeVarchar) {
  195. size = size - 1
  196. }
  197. } else {
  198. if !(len(table.ColumnAliases) != 0 || len(table.Columns) == 0) {
  199. size = 0
  200. }
  201. }
  202. table.ClusteringColumns = make([]*ColumnMetadata, size)
  203. for i := range table.ClusteringColumns {
  204. var alias string
  205. if len(table.ColumnAliases) > i {
  206. alias = table.ColumnAliases[i]
  207. } else if i == 0 {
  208. alias = DEFAULT_COLUMN_ALIAS
  209. } else {
  210. alias = DEFAULT_COLUMN_ALIAS + strconv.Itoa(i+1)
  211. }
  212. order := ASC
  213. if comparatorParsed.reversed[i] {
  214. order = DESC
  215. }
  216. column := &ColumnMetadata{
  217. Keyspace: table.Keyspace,
  218. Table: table.Name,
  219. Name: alias,
  220. Type: comparatorParsed.types[i],
  221. Order: order,
  222. Kind: CLUSTERING_KEY,
  223. ComponentIndex: i,
  224. }
  225. table.ClusteringColumns[i] = column
  226. table.Columns[alias] = column
  227. }
  228. if size != len(comparatorParsed.types)-1 {
  229. alias := DEFAULT_VALUE_ALIAS
  230. if len(table.ValueAlias) > 0 {
  231. alias = table.ValueAlias
  232. }
  233. // decode the default validator
  234. defaultValidatorParsed := parseType(table.DefaultValidator)
  235. column := &ColumnMetadata{
  236. Keyspace: table.Keyspace,
  237. Table: table.Name,
  238. Name: alias,
  239. Type: defaultValidatorParsed.types[0],
  240. Kind: REGULAR,
  241. }
  242. table.Columns[alias] = column
  243. }
  244. }
  245. }
  246. // The simpler compile case for V2+ protocol
  247. func compileV2Metadata(tables []TableMetadata) {
  248. for i := range tables {
  249. table := &tables[i]
  250. partitionColumnCount := countColumnsOfKind(table.Columns, PARTITION_KEY)
  251. table.PartitionKey = make([]*ColumnMetadata, partitionColumnCount)
  252. clusteringColumnCount := countColumnsOfKind(table.Columns, CLUSTERING_KEY)
  253. table.ClusteringColumns = make([]*ColumnMetadata, clusteringColumnCount)
  254. for _, column := range table.Columns {
  255. if column.Kind == PARTITION_KEY {
  256. table.PartitionKey[column.ComponentIndex] = column
  257. } else if column.Kind == CLUSTERING_KEY {
  258. table.ClusteringColumns[column.ComponentIndex] = column
  259. }
  260. }
  261. }
  262. }
  263. func countColumnsOfKind(columns map[string]*ColumnMetadata, kind string) int {
  264. count := 0
  265. for _, column := range columns {
  266. if column.Kind == kind {
  267. count++
  268. }
  269. }
  270. return count
  271. }
  272. // query only for the keyspace metadata for the specified keyspace
  273. func getKeyspaceMetadata(
  274. session *Session,
  275. keyspaceName string,
  276. ) (*KeyspaceMetadata, error) {
  277. query := session.Query(
  278. `
  279. SELECT durable_writes, strategy_class, strategy_options
  280. FROM system.schema_keyspaces
  281. WHERE keyspace_name = ?
  282. `,
  283. keyspaceName,
  284. )
  285. // Set a routing key to avoid GetRoutingKey from computing the routing key
  286. // TODO use a separate connection (pool) for system keyspace queries.
  287. query.RoutingKey([]byte{})
  288. keyspace := &KeyspaceMetadata{Name: keyspaceName}
  289. var strategyOptionsJSON []byte
  290. err := query.Scan(
  291. &keyspace.DurableWrites,
  292. &keyspace.StrategyClass,
  293. &strategyOptionsJSON,
  294. )
  295. if err != nil {
  296. return nil, fmt.Errorf("Error querying keyspace schema: %v", err)
  297. }
  298. err = json.Unmarshal(strategyOptionsJSON, &keyspace.StrategyOptions)
  299. if err != nil {
  300. return nil, fmt.Errorf(
  301. "Invalid JSON value '%s' as strategy_options for in keyspace '%s': %v",
  302. strategyOptionsJSON, keyspace.Name, err,
  303. )
  304. }
  305. return keyspace, nil
  306. }
  307. // query for only the table metadata in the specified keyspace
  308. func getTableMetadata(
  309. session *Session,
  310. keyspaceName string,
  311. ) ([]TableMetadata, error) {
  312. query := session.Query(
  313. `
  314. SELECT
  315. columnfamily_name,
  316. key_validator,
  317. comparator,
  318. default_validator,
  319. key_aliases,
  320. column_aliases,
  321. value_alias
  322. FROM system.schema_columnfamilies
  323. WHERE keyspace_name = ?
  324. `,
  325. keyspaceName,
  326. )
  327. // Set a routing key to avoid GetRoutingKey from computing the routing key
  328. // TODO use a separate connection (pool) for system keyspace queries.
  329. query.RoutingKey([]byte{})
  330. iter := query.Iter()
  331. tables := []TableMetadata{}
  332. table := TableMetadata{Keyspace: keyspaceName}
  333. var keyAliasesJSON []byte
  334. var columnAliasesJSON []byte
  335. for iter.Scan(
  336. &table.Name,
  337. &table.KeyValidator,
  338. &table.Comparator,
  339. &table.DefaultValidator,
  340. &keyAliasesJSON,
  341. &columnAliasesJSON,
  342. &table.ValueAlias,
  343. ) {
  344. var err error
  345. // decode the key aliases
  346. if keyAliasesJSON != nil {
  347. table.KeyAliases = []string{}
  348. err = json.Unmarshal(keyAliasesJSON, &table.KeyAliases)
  349. if err != nil {
  350. iter.Close()
  351. return nil, fmt.Errorf(
  352. "Invalid JSON value '%s' as key_aliases for in table '%s': %v",
  353. keyAliasesJSON, table.Name, err,
  354. )
  355. }
  356. }
  357. // decode the column aliases
  358. if columnAliasesJSON != nil {
  359. table.ColumnAliases = []string{}
  360. err = json.Unmarshal(columnAliasesJSON, &table.ColumnAliases)
  361. if err != nil {
  362. iter.Close()
  363. return nil, fmt.Errorf(
  364. "Invalid JSON value '%s' as column_aliases for in table '%s': %v",
  365. columnAliasesJSON, table.Name, err,
  366. )
  367. }
  368. }
  369. tables = append(tables, table)
  370. table = TableMetadata{Keyspace: keyspaceName}
  371. }
  372. err := iter.Close()
  373. if err != nil && err != ErrNotFound {
  374. return nil, fmt.Errorf("Error querying table schema: %v", err)
  375. }
  376. return tables, nil
  377. }
  378. // query for only the table metadata in the specified keyspace
  379. func getColumnMetadata(
  380. session *Session,
  381. keyspaceName string,
  382. ) ([]ColumnMetadata, error) {
  383. // Deal with differences in protocol versions
  384. var stmt string
  385. var scan func(*Iter, *ColumnMetadata, *[]byte) bool
  386. if session.cfg.ProtoVersion == 1 {
  387. // V1 does not support the type column, and all returned rows are
  388. // of kind "regular".
  389. stmt = `
  390. SELECT
  391. columnfamily_name,
  392. column_name,
  393. component_index,
  394. validator,
  395. index_name,
  396. index_type,
  397. index_options
  398. FROM system.schema_columns
  399. WHERE keyspace_name = ?
  400. `
  401. scan = func(
  402. iter *Iter,
  403. column *ColumnMetadata,
  404. indexOptionsJSON *[]byte,
  405. ) bool {
  406. // all columns returned by V1 are regular
  407. column.Kind = REGULAR
  408. return iter.Scan(
  409. &column.Table,
  410. &column.Name,
  411. &column.ComponentIndex,
  412. &column.Validator,
  413. &column.Index.Name,
  414. &column.Index.Type,
  415. &indexOptionsJSON,
  416. )
  417. }
  418. } else {
  419. // V2+ supports the type column
  420. stmt = `
  421. SELECT
  422. columnfamily_name,
  423. column_name,
  424. component_index,
  425. validator,
  426. index_name,
  427. index_type,
  428. index_options,
  429. type
  430. FROM system.schema_columns
  431. WHERE keyspace_name = ?
  432. `
  433. scan = func(
  434. iter *Iter,
  435. column *ColumnMetadata,
  436. indexOptionsJSON *[]byte,
  437. ) bool {
  438. return iter.Scan(
  439. &column.Table,
  440. &column.Name,
  441. &column.ComponentIndex,
  442. &column.Validator,
  443. &column.Index.Name,
  444. &column.Index.Type,
  445. &indexOptionsJSON,
  446. &column.Kind,
  447. )
  448. }
  449. }
  450. // get the columns metadata
  451. columns := []ColumnMetadata{}
  452. column := ColumnMetadata{Keyspace: keyspaceName}
  453. var indexOptionsJSON []byte
  454. query := session.Query(stmt, keyspaceName)
  455. // Set a routing key to avoid GetRoutingKey from computing the routing key
  456. // TODO use a separate connection (pool) for system keyspace queries.
  457. query.RoutingKey([]byte{})
  458. iter := query.Iter()
  459. for scan(iter, &column, &indexOptionsJSON) {
  460. var err error
  461. // decode the index options
  462. if indexOptionsJSON != nil {
  463. err = json.Unmarshal(indexOptionsJSON, &column.Index.Options)
  464. if err != nil {
  465. iter.Close()
  466. return nil, fmt.Errorf(
  467. "Invalid JSON value '%s' as index_options for column '%s' in table '%s': %v",
  468. indexOptionsJSON,
  469. column.Name,
  470. column.Table,
  471. err,
  472. )
  473. }
  474. }
  475. columns = append(columns, column)
  476. column = ColumnMetadata{Keyspace: keyspaceName}
  477. }
  478. err := iter.Close()
  479. if err != nil && err != ErrNotFound {
  480. return nil, fmt.Errorf("Error querying column schema: %v", err)
  481. }
  482. return columns, nil
  483. }
  484. // type definition parser state
  485. type typeParser struct {
  486. input string
  487. index int
  488. }
  489. // the type definition parser result
  490. type typeParserResult struct {
  491. isComposite bool
  492. types []TypeInfo
  493. reversed []bool
  494. collections map[string]TypeInfo
  495. }
  496. // Parse the type definition used for validator and comparator schema data
  497. func parseType(def string) typeParserResult {
  498. parser := &typeParser{input: def}
  499. return parser.parse()
  500. }
  501. const (
  502. REVERSED_TYPE = "org.apache.cassandra.db.marshal.ReversedType"
  503. COMPOSITE_TYPE = "org.apache.cassandra.db.marshal.CompositeType"
  504. COLLECTION_TYPE = "org.apache.cassandra.db.marshal.ColumnToCollectionType"
  505. LIST_TYPE = "org.apache.cassandra.db.marshal.ListType"
  506. SET_TYPE = "org.apache.cassandra.db.marshal.SetType"
  507. MAP_TYPE = "org.apache.cassandra.db.marshal.MapType"
  508. )
  509. // represents a class specification in the type def AST
  510. type typeParserClassNode struct {
  511. name string
  512. params []typeParserParamNode
  513. // this is the segment of the input string that defined this node
  514. input string
  515. }
  516. // represents a class parameter in the type def AST
  517. type typeParserParamNode struct {
  518. name *string
  519. class typeParserClassNode
  520. }
  521. func (t *typeParser) parse() typeParserResult {
  522. // parse the AST
  523. ast, ok := t.parseClassNode()
  524. if !ok {
  525. // treat this is a custom type
  526. return typeParserResult{
  527. isComposite: false,
  528. types: []TypeInfo{
  529. NativeType{
  530. typ: TypeCustom,
  531. custom: t.input,
  532. },
  533. },
  534. reversed: []bool{false},
  535. collections: nil,
  536. }
  537. }
  538. // interpret the AST
  539. if strings.HasPrefix(ast.name, COMPOSITE_TYPE) {
  540. count := len(ast.params)
  541. // look for a collections param
  542. last := ast.params[count-1]
  543. collections := map[string]TypeInfo{}
  544. if strings.HasPrefix(last.class.name, COLLECTION_TYPE) {
  545. count--
  546. for _, param := range last.class.params {
  547. // decode the name
  548. var name string
  549. decoded, err := hex.DecodeString(*param.name)
  550. if err != nil {
  551. log.Printf(
  552. "Error parsing type '%s', contains collection name '%s' with an invalid format: %v",
  553. t.input,
  554. *param.name,
  555. err,
  556. )
  557. // just use the provided name
  558. name = *param.name
  559. } else {
  560. name = string(decoded)
  561. }
  562. collections[name] = param.class.asTypeInfo()
  563. }
  564. }
  565. types := make([]TypeInfo, count)
  566. reversed := make([]bool, count)
  567. for i, param := range ast.params[:count] {
  568. class := param.class
  569. reversed[i] = strings.HasPrefix(class.name, REVERSED_TYPE)
  570. if reversed[i] {
  571. class = class.params[0].class
  572. }
  573. types[i] = class.asTypeInfo()
  574. }
  575. return typeParserResult{
  576. isComposite: true,
  577. types: types,
  578. reversed: reversed,
  579. collections: collections,
  580. }
  581. } else {
  582. // not composite, so one type
  583. class := *ast
  584. reversed := strings.HasPrefix(class.name, REVERSED_TYPE)
  585. if reversed {
  586. class = class.params[0].class
  587. }
  588. typeInfo := class.asTypeInfo()
  589. return typeParserResult{
  590. isComposite: false,
  591. types: []TypeInfo{typeInfo},
  592. reversed: []bool{reversed},
  593. }
  594. }
  595. }
  596. func (class *typeParserClassNode) asTypeInfo() TypeInfo {
  597. if strings.HasPrefix(class.name, LIST_TYPE) {
  598. elem := class.params[0].class.asTypeInfo()
  599. return CollectionType{
  600. NativeType: NativeType{
  601. typ: TypeList,
  602. },
  603. Elem: elem,
  604. }
  605. }
  606. if strings.HasPrefix(class.name, SET_TYPE) {
  607. elem := class.params[0].class.asTypeInfo()
  608. return CollectionType{
  609. NativeType: NativeType{
  610. typ: TypeSet,
  611. },
  612. Elem: elem,
  613. }
  614. }
  615. if strings.HasPrefix(class.name, MAP_TYPE) {
  616. key := class.params[0].class.asTypeInfo()
  617. elem := class.params[1].class.asTypeInfo()
  618. return CollectionType{
  619. NativeType: NativeType{
  620. typ: TypeMap,
  621. },
  622. Key: key,
  623. Elem: elem,
  624. }
  625. }
  626. // must be a simple type or custom type
  627. info := NativeType{typ: getApacheCassandraType(class.name)}
  628. if info.typ == TypeCustom {
  629. // add the entire class definition
  630. info.custom = class.input
  631. }
  632. return info
  633. }
  634. // CLASS := ID [ PARAMS ]
  635. func (t *typeParser) parseClassNode() (node *typeParserClassNode, ok bool) {
  636. t.skipWhitespace()
  637. startIndex := t.index
  638. name, ok := t.nextIdentifier()
  639. if !ok {
  640. return nil, false
  641. }
  642. params, ok := t.parseParamNodes()
  643. if !ok {
  644. return nil, false
  645. }
  646. endIndex := t.index
  647. node = &typeParserClassNode{
  648. name: name,
  649. params: params,
  650. input: t.input[startIndex:endIndex],
  651. }
  652. return node, true
  653. }
  654. // PARAMS := "(" PARAM { "," PARAM } ")"
  655. // PARAM := [ PARAM_NAME ":" ] CLASS
  656. // PARAM_NAME := ID
  657. func (t *typeParser) parseParamNodes() (params []typeParserParamNode, ok bool) {
  658. t.skipWhitespace()
  659. // the params are optional
  660. if t.index == len(t.input) || t.input[t.index] != '(' {
  661. return nil, true
  662. }
  663. params = []typeParserParamNode{}
  664. // consume the '('
  665. t.index++
  666. t.skipWhitespace()
  667. for t.input[t.index] != ')' {
  668. // look for a named param, but if no colon, then we want to backup
  669. backupIndex := t.index
  670. // name will be a hex encoded version of a utf-8 string
  671. name, ok := t.nextIdentifier()
  672. if !ok {
  673. return nil, false
  674. }
  675. hasName := true
  676. // TODO handle '=>' used for DynamicCompositeType
  677. t.skipWhitespace()
  678. if t.input[t.index] == ':' {
  679. // there is a name for this parameter
  680. // consume the ':'
  681. t.index++
  682. t.skipWhitespace()
  683. } else {
  684. // no name, backup
  685. hasName = false
  686. t.index = backupIndex
  687. }
  688. // parse the next full parameter
  689. classNode, ok := t.parseClassNode()
  690. if !ok {
  691. return nil, false
  692. }
  693. if hasName {
  694. params = append(
  695. params,
  696. typeParserParamNode{name: &name, class: *classNode},
  697. )
  698. } else {
  699. params = append(
  700. params,
  701. typeParserParamNode{class: *classNode},
  702. )
  703. }
  704. t.skipWhitespace()
  705. if t.input[t.index] == ',' {
  706. // consume the comma
  707. t.index++
  708. t.skipWhitespace()
  709. }
  710. }
  711. // consume the ')'
  712. t.index++
  713. return params, true
  714. }
  715. func (t *typeParser) skipWhitespace() {
  716. for t.index < len(t.input) && isWhitespaceChar(t.input[t.index]) {
  717. t.index++
  718. }
  719. }
  720. func isWhitespaceChar(c byte) bool {
  721. return c == ' ' || c == '\n' || c == '\t'
  722. }
  723. // ID := LETTER { LETTER }
  724. // LETTER := "0"..."9" | "a"..."z" | "A"..."Z" | "-" | "+" | "." | "_" | "&"
  725. func (t *typeParser) nextIdentifier() (id string, found bool) {
  726. startIndex := t.index
  727. for t.index < len(t.input) && isIdentifierChar(t.input[t.index]) {
  728. t.index++
  729. }
  730. if startIndex == t.index {
  731. return "", false
  732. }
  733. return t.input[startIndex:t.index], true
  734. }
  735. func isIdentifierChar(c byte) bool {
  736. return (c >= '0' && c <= '9') ||
  737. (c >= 'a' && c <= 'z') ||
  738. (c >= 'A' && c <= 'Z') ||
  739. c == '-' ||
  740. c == '+' ||
  741. c == '.' ||
  742. c == '_' ||
  743. c == '&'
  744. }