metadata.go 19 KB

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