From 6c4bccb44442993a0131d0b7fa353c9cf35426e4 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Fri, 29 Jan 2021 18:46:34 -0400 Subject: [PATCH 1/3] sql: add schema_name,table_id to crdb_internal.ranges ... and crdb_internal.ranges_no_leases This commit adds schema_name to crdb_internal.ranges and crdb_internal.ranges_no_leases to ensure that it's possible to disambiguate between ranges that are contained by a table with the same name in two different user-defined schemas. In addition, it also adds the table_id column which allows unambiguous lookups of ranges for a given table id. This will also enable making a virtual index on the table_id column later, which should be a nice win for some introspection commands. Release note (sql change): add the schema_name and table_id columns to the crdb_internal.ranges and crdb_internal.ranges_no_leases virtual tables. --- pkg/sql/crdb_internal.go | 27 +++++++++++++++++-- .../testdata/logic_test/crdb_internal | 8 +++--- 2 files changed, 29 insertions(+), 6 deletions(-) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 6ce44557a3c1..a43cd8300df0 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2494,7 +2494,9 @@ CREATE VIEW crdb_internal.ranges AS SELECT start_pretty, end_key, end_pretty, + table_id, database_name, + schema_name, table_name, index_name, replicas, @@ -2512,7 +2514,9 @@ FROM crdb_internal.ranges_no_leases {Name: "start_pretty", Typ: types.String}, {Name: "end_key", Typ: types.Bytes}, {Name: "end_pretty", Typ: types.String}, + {Name: "table_id", Typ: types.Int}, {Name: "database_name", Typ: types.String}, + {Name: "schema_name", Typ: types.String}, {Name: "table_name", Typ: types.String}, {Name: "index_name", Typ: types.String}, {Name: "replicas", Typ: types.Int2Vector}, @@ -2537,7 +2541,9 @@ CREATE TABLE crdb_internal.ranges_no_leases ( start_pretty STRING NOT NULL, end_key BYTES NOT NULL, end_pretty STRING NOT NULL, + table_id INT NOT NULL, database_name STRING NOT NULL, + schema_name STRING NOT NULL, table_name STRING NOT NULL, index_name STRING NOT NULL, replicas INT[] NOT NULL, @@ -2557,13 +2563,16 @@ CREATE TABLE crdb_internal.ranges_no_leases ( // TODO(knz): maybe this could use internalLookupCtx. dbNames := make(map[uint32]string) tableNames := make(map[uint32]string) + schemaNames := make(map[uint32]string) indexNames := make(map[uint32]map[uint32]string) + schemaParents := make(map[uint32]uint32) parents := make(map[uint32]uint32) for _, desc := range descs { id := uint32(desc.GetID()) switch desc := desc.(type) { case catalog.TableDescriptor: parents[id] = uint32(desc.GetParentID()) + schemaParents[id] = uint32(desc.GetParentSchemaID()) tableNames[id] = desc.GetName() indexNames[id] = make(map[uint32]string) for _, idx := range desc.PublicNonPrimaryIndexes() { @@ -2571,6 +2580,8 @@ CREATE TABLE crdb_internal.ranges_no_leases ( } case *dbdesc.Immutable: dbNames[id] = desc.GetName() + case *schemadesc.Immutable: + schemaNames[id] = desc.GetName() } } ranges, err := kvclient.ScanMetaKVs(ctx, p.txn, roachpb.Span{ @@ -2637,8 +2648,18 @@ CREATE TABLE crdb_internal.ranges_no_leases ( } } - var dbName, tableName, indexName string - if _, tableID, err := p.ExecCfg().Codec.DecodeTablePrefix(desc.StartKey.AsRawKey()); err == nil { + var dbName, schemaName, tableName, indexName string + var tableID uint32 + if _, tableID, err = p.ExecCfg().Codec.DecodeTablePrefix(desc.StartKey.AsRawKey()); err == nil { + schemaParent := schemaParents[tableID] + if schemaParent != 0 { + schemaName = schemaNames[schemaParent] + } else { + // This case shouldn't happen - all schema ids should be available in the + // schemaParents map. If it's not, just assume the name of the schema + // is public to avoid problems. + schemaName = string(tree.PublicSchemaName) + } parent := parents[tableID] if parent != 0 { tableName = tableNames[tableID] @@ -2662,7 +2683,9 @@ CREATE TABLE crdb_internal.ranges_no_leases ( tree.NewDString(keys.PrettyPrint(nil /* valDirs */, desc.StartKey.AsRawKey())), tree.NewDBytes(tree.DBytes(desc.EndKey)), tree.NewDString(keys.PrettyPrint(nil /* valDirs */, desc.EndKey.AsRawKey())), + tree.NewDInt(tree.DInt(tableID)), tree.NewDString(dbName), + tree.NewDString(schemaName), tree.NewDString(tableName), tree.NewDString(indexName), votersArr, diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index ee7c7b93e6f8..8eac7f36f71b 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -255,15 +255,15 @@ SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0 ---- trace_id parent_span_id span_id goroutine_id start_time duration operation -query ITTTTTTTTTTTTI colnames +query ITTTTITTTTTTTTTI colnames SELECT * FROM crdb_internal.ranges WHERE range_id < 0 ---- -range_id start_key start_pretty end_key end_pretty database_name table_name index_name replicas replica_localities learner_replicas split_enforced_until lease_holder range_size +range_id start_key start_pretty end_key end_pretty table_id database_name schema_name table_name index_name replicas replica_localities learner_replicas split_enforced_until lease_holder range_size -query ITTTTTTTTTTT colnames +query ITTTTITTTTTTTT colnames SELECT * FROM crdb_internal.ranges_no_leases WHERE range_id < 0 ---- -range_id start_key start_pretty end_key end_pretty database_name table_name index_name replicas replica_localities learner_replicas split_enforced_until +range_id start_key start_pretty end_key end_pretty table_id database_name schema_name table_name index_name replicas replica_localities learner_replicas split_enforced_until statement ok INSERT INTO system.zones (id, config) VALUES From 306d2e9939fe85ed98eac48c1a4b9c120407653b Mon Sep 17 00:00:00 2001 From: Alex Lunev Date: Fri, 12 Feb 2021 14:03:09 -0800 Subject: [PATCH 2/3] kvserver: improve handling for removal of a replica, when multiple replicas already exist on the same node Fixes #60545 The allocator in some cases allows for a range to have a replica on multiple stores of the same node. If that happens, it should allow itself to fix the situation by removing one of the offending replicas. This was only half working due to an ordering problem in how the replicas appeared in the descriptor. It could remove the first replica, but not the second one. Release note (bug fix): 20.2 introduced an ability to rebalance replicas between multiple stores on the same node. This change fixed a problem with that feature, where ocassionaly an intra-node rebalance would fail and a range would get stuck permanently under replicated. --- pkg/kv/kvserver/replica_command.go | 41 +++++++++++++++++++++++-- pkg/kv/kvserver/replica_command_test.go | 14 ++++++--- 2 files changed, 48 insertions(+), 7 deletions(-) diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 053186a5faec..e1e38c3fdfe1 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -1158,6 +1158,27 @@ func maybeLeaveAtomicChangeReplicasAndRemoveLearners( return desc, nil } +func validateReplicationChangesMultipleReplicasOnTheSameNode( + byStoreID map[roachpb.StoreID]roachpb.ReplicationChange, + rDescsForNode []roachpb.ReplicaDescriptor, +) error { + if len(byStoreID) != 1 { + return errors.Errorf( + "An unexpected number of changes %s for range %s, the only valid operation when there are already multiple replicas on the same node is removal", byStoreID, rDescsForNode) + } + for _, rDesc := range rDescsForNode { + chg, ok := byStoreID[rDesc.StoreID] + if ok { + if !chg.ChangeType.IsRemoval() { + return errors.Errorf( + "Expected replica to be removed from %v instead got %v.", rDesc, chg) + } + return nil + } + } + return errors.Errorf("Expected a removal of one of the replicas in %s, instead got %s", rDescsForNode, byStoreID) +} + func validateReplicationChanges( desc *roachpb.RangeDescriptor, chgs roachpb.ReplicationChanges, ) error { @@ -1188,15 +1209,29 @@ func validateReplicationChanges( byStoreID[chg.Target.StoreID] = chg } + descriptorsByNodeID := make(map[roachpb.NodeID][]roachpb.ReplicaDescriptor, len(desc.Replicas().Descriptors())) + for _, rDesc := range desc.Replicas().Descriptors() { + descriptorsByNodeID[rDesc.NodeID] = append(descriptorsByNodeID[rDesc.NodeID], rDesc) + } + // Then, check that we're not adding a second replica on nodes that already // have one, or "re-add" an existing replica. We delete from byNodeAndStoreID so that // after this loop, it contains only Nodes that we haven't seen in desc. - for _, rDesc := range desc.Replicas().Descriptors() { - byStoreID, ok := byNodeAndStoreID[rDesc.NodeID] + for nodeID, rDescsForNode := range descriptorsByNodeID { + byStoreID, ok := byNodeAndStoreID[nodeID] if !ok { continue } - delete(byNodeAndStoreID, rDesc.NodeID) + delete(byNodeAndStoreID, nodeID) + // The only valid thing to do when we already have multiple replicas on the + // same node is to remove a replica. + if len(rDescsForNode) > 1 { + if err := validateReplicationChangesMultipleReplicasOnTheSameNode(byStoreID, rDescsForNode); err != nil { + return err + } + continue + } + rDesc := rDescsForNode[0] if len(byStoreID) == 2 { chg, k := byStoreID[rDesc.StoreID] // We should be removing the replica from the existing store during a diff --git a/pkg/kv/kvserver/replica_command_test.go b/pkg/kv/kvserver/replica_command_test.go index fd7b20221983..a68ec5feec50 100644 --- a/pkg/kv/kvserver/replica_command_test.go +++ b/pkg/kv/kvserver/replica_command_test.go @@ -198,7 +198,7 @@ func TestValidateReplicationChanges(t *testing.T) { InternalReplicas: []roachpb.ReplicaDescriptor{ {NodeID: 1, StoreID: 1}, {NodeID: 2, StoreID: 2}, - {NodeID: 1, StoreID: 2, Type: &learnerType}, + {NodeID: 1, StoreID: 3, Type: &learnerType}, }, } err = validateReplicationChanges(descRebalancing, roachpb.ReplicationChanges{ @@ -206,20 +206,26 @@ func TestValidateReplicationChanges(t *testing.T) { }) require.NoError(t, err) - // Test Case 15: Do an add while rebalancing within a node + // Test Case 15: same as 14 but remove the second node + err = validateReplicationChanges(descRebalancing, roachpb.ReplicationChanges{ + {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 3}}, + }) + require.NoError(t, err) + + // Test Case 16: Do an add while rebalancing within a node err = validateReplicationChanges(descRebalancing, roachpb.ReplicationChanges{ {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 3, StoreID: 3}}, }) require.NoError(t, err) - // Test Case 16: Remove/Add within a node is not allowed, since we expect Add/Remove + // Test Case 17: Remove/Add within a node is not allowed, since we expect Add/Remove err = validateReplicationChanges(desc, roachpb.ReplicationChanges{ {ChangeType: roachpb.REMOVE_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 1}}, {ChangeType: roachpb.ADD_VOTER, Target: roachpb.ReplicationTarget{NodeID: 1, StoreID: 2}}, }) require.Regexp(t, "can only add-remove a replica within a node, but got ", err) - // Test Case 17: We are rebalancing within a node and have only one replica + // Test Case 18: We are rebalancing within a node and have only one replica descSingle := &roachpb.RangeDescriptor{ InternalReplicas: []roachpb.ReplicaDescriptor{ {NodeID: 1, StoreID: 1}, From 0abdc79790cbabaf49966a0040d71d32318cc932 Mon Sep 17 00:00:00 2001 From: Andy Yang Date: Sun, 14 Feb 2021 00:10:39 -0500 Subject: [PATCH 3/3] geo/wkt: simplify parser grammar and improve error messages This patch simplifies the yacc grammar for the WKT parser and also improves the error messages for mixed dimensionality problems. Release note: None --- pkg/geo/wkt/lex.go | 145 +++++- pkg/geo/wkt/wkt.y | 840 +++++++++++++----------------- pkg/geo/wkt/wkt_generated.go | 960 +++++++++++++---------------------- pkg/geo/wkt/wkt_test.go | 57 ++- 4 files changed, 906 insertions(+), 1096 deletions(-) diff --git a/pkg/geo/wkt/lex.go b/pkg/geo/wkt/lex.go index 913fa0290ac1..9b4a2c71995a 100644 --- a/pkg/geo/wkt/lex.go +++ b/pkg/geo/wkt/lex.go @@ -36,21 +36,28 @@ type ParseError struct { problem string pos int str string + hint string } func (e *ParseError) Error() string { - return fmt.Sprintf("%s at pos %d\n%s\n%s^", e.problem, e.pos, e.str, strings.Repeat(" ", e.pos)) + // TODO(ayang): print only relevant line (with line and pos no) instead of entire input + err := fmt.Sprintf("%s at pos %d\n%s\n%s^", e.problem, e.pos, e.str, strings.Repeat(" ", e.pos)) + if e.hint != "" { + err += fmt.Sprintf("\nHINT: %s", e.hint) + } + return err } // Constant expected by parser when lexer reaches EOF. const eof = 0 type wktLex struct { - line string - pos int - lastPos int - ret geom.T - lastErr error + line string + pos int + lastPos int + ret geom.T + curLayout geom.Layout + lastErr error } // Lex lexes a token from the input. @@ -224,10 +231,136 @@ func (l *wktLex) trimLeft() { } } +func getDefaultLayoutForStride(stride int) geom.Layout { + switch stride { + case 2: + return geom.XY + case 3: + return geom.XYZ + case 4: + return geom.XYZM + default: + // This should never happen. + panic("unsupported stride") + } +} + +func (l *wktLex) validateStrideAndSetLayoutIfNoLayout(stride int) bool { + if !l.validateStride(stride) { + return false + } + l.setLayoutIfNoLayout(getDefaultLayoutForStride(stride)) + return true +} + +func (l *wktLex) validateStride(stride int) bool { + if !l.isValidStrideForLayout(stride) { + l.setIncorrectStrideError(stride, "") + return false + } + return true +} + +func (l *wktLex) isValidStrideForLayout(stride int) bool { + switch l.curLayout { + case geom.NoLayout: + return true + case geom.XY: + return stride == 2 + case geom.XYM: + return stride == 3 + case geom.XYZ: + return stride == 3 + case geom.XYZM: + return stride == 4 + default: + // This should never happen. + panic("unknown geom.Layout") + } +} + +func (l *wktLex) setLayout(layout geom.Layout) bool { + if layout == l.curLayout { + return true + } + if l.curLayout != geom.NoLayout { + l.setIncorrectLayoutError(layout, "") + return false + } + l.setLayoutIfNoLayout(layout) + return true +} + +func (l *wktLex) setLayoutEmptyInCollection() bool { + if l.curLayout == geom.XY { + return true + } + if l.curLayout == geom.NoLayout { + l.curLayout = geom.XY + return true + } + l.setIncorrectLayoutError(geom.XY, "EMPTY is XY layout in base geometry type collection") + return false +} + +func (l *wktLex) setLayoutIfNoLayout(layout geom.Layout) { + switch l.curLayout { + case geom.NoLayout: + l.curLayout = layout + case geom.XY, geom.XYM, geom.XYZ, geom.XYZM: + break + default: + // This should never happen. + panic("unknown geom.Layout") + } +} + func (l *wktLex) setLexError(expectedTokType string) { l.lastErr = &LexError{expectedTokType: expectedTokType, pos: l.lastPos, str: l.line} } +func getLayoutName(layout geom.Layout) string { + switch layout { + case geom.XY: + return "XY" + case geom.XYM: + return "XYM" + case geom.XYZ: + return "XYZ" + case geom.XYZM: + return "XYZM" + default: + // This should never happen. + panic("unknown geom.Layout") + } +} + +func (l *wktLex) setIncorrectStrideError(incorrectStride int, hint string) { + problem := fmt.Sprintf("mixed dimensionality, parsed layout is %s so expecting %d coords but got %d coords", + getLayoutName(l.curLayout), l.curLayout.Stride(), incorrectStride) + l.setParseError(problem, hint) +} + +func (l *wktLex) setIncorrectLayoutError(incorrectLayout geom.Layout, hint string) { + problem := fmt.Sprintf("mixed dimensionality, parsed layout is %s but encountered layout of %s", + getLayoutName(l.curLayout), getLayoutName(incorrectLayout)) + l.setParseError(problem, hint) +} + +func (l *wktLex) setParseError(problem string, hint string) { + // Lex errors take precedence. + if l.lastErr != nil { + return + } + errProblem := "syntax error: " + problem + l.lastErr = &ParseError{ + problem: errProblem, + pos: l.lastPos, + str: l.line, + hint: hint, + } +} + func (l *wktLex) Error(s string) { // NB: Lex errors are set in the Lex function. if l.lastErr == nil { diff --git a/pkg/geo/wkt/wkt.y b/pkg/geo/wkt/wkt.y index 0a8282fed024..84acb046f5be 100644 --- a/pkg/geo/wkt/wkt.y +++ b/pkg/geo/wkt/wkt.y @@ -16,7 +16,7 @@ import "github.com/twpayne/go-geom" func isValidLineString(wktlex wktLexer, flatCoords []float64, stride int) bool { if len(flatCoords) < 2 * stride { - wktlex.(*wktLex).Error("syntax error: non-empty linestring with only one point") + wktlex.(*wktLex).setParseError("non-empty linestring with only one point", "minimum number of points is 2") return false } return true @@ -24,12 +24,12 @@ func isValidLineString(wktlex wktLexer, flatCoords []float64, stride int) bool { func isValidPolygonRing(wktlex wktLexer, flatCoords []float64, stride int) bool { if len(flatCoords) < 4 * stride { - wktlex.(*wktLex).Error("syntax error: polygon ring doesn't have enough points") + wktlex.(*wktLex).setParseError("polygon ring doesn't have enough points", "minimum number of points is 4") return false } for i := 0; i < stride; i++ { if flatCoords[i] != flatCoords[len(flatCoords)-stride+i] { - wktlex.(*wktLex).Error("syntax error: polygon ring not closed") + wktlex.(*wktLex).setParseError("polygon ring not closed", "ensure first and last point are the same") return false } } @@ -100,6 +100,7 @@ func appendMultiPolygonFlatCoordsRepr( multiPolyFlatRepr multiPolygonFlatCoordsRepr } +// Tokens %token POINT POINTM POINTZ POINTZM %token LINESTRING LINESTRINGM LINESTRINGZ LINESTRINGZM %token POLYGON POLYGONM POLYGONZ POLYGONZM @@ -110,47 +111,63 @@ func appendMultiPolygonFlatCoordsRepr( //%token GEOMETRYCOLLECTION %token NUM +// Geometries %type geometry %type point linestring polygon multipoint multilinestring multipolygon -// TODO(ayang) reorganize the list of %type statements -%type two_coords three_coords four_coords -%type two_coords_point_with_parens three_coords_point_with_parens four_coords_point_with_parens -%type two_coords_list three_coords_list four_coords_list -%type two_coords_list_with_parens three_coords_list_with_parens four_coords_list_with_parens -%type two_coords_line three_coords_line four_coords_line -%type two_coords_ring three_coords_ring four_coords_ring -%type two_coords_ring_list three_coords_ring_list four_coords_ring_list -%type two_coords_point three_coords_point four_coords_point -%type three_coords_point_list four_coords_point_list -%type empty_point empty_line_flat_repr -%type two_coords_point_allowing_empty three_coords_point_allowing_empty four_coords_point_allowing_empty -%type two_coords_point_list_allowing_empty_points -%type three_coords_point_list_allowing_empty_points -%type four_coords_point_list_allowing_empty_points -%type two_coords_line_flat_repr three_coords_line_flat_repr four_coords_line_flat_repr -%type two_coords_line_allowing_empty three_coords_line_allowing_empty four_coords_line_allowing_empty -%type three_coords_line_list four_coords_line_list -%type two_coords_line_list_allowing_empty_lines -%type three_coords_line_list_allowing_empty_lines -%type four_coords_line_list_allowing_empty_lines -%type two_coords_polygon three_coords_polygon four_coords_polygon empty_polygon - -%type two_coords_polygon_multi_poly_flat_repr -%type three_coords_polygon_multi_poly_flat_repr -%type four_coords_polygon_multi_poly_flat_repr -%type empty_polygon_multi_poly_flat_repr - -%type three_coords_polygon_list -%type four_coords_polygon_list - -%type two_coords_polygon_allowing_empty -%type three_coords_polygon_allowing_empty -%type four_coords_polygon_allowing_empty - -%type two_coords_polygon_list_allowing_empty_polygons -%type three_coords_polygon_list_allowing_empty_polygons -%type four_coords_polygon_list_allowing_empty_polygons +// Empty representation +%type empty +%type empty_in_base_type_collection +%type flat_coords_empty + +// Points +%type flat_coords +%type flat_coords_point +%type flat_coords_point_with_parens + +// LineStrings +%type flat_coords_point_list +%type flat_coords_point_list_with_parens +%type flat_coords_linestring + +// Polygons +%type flat_coords_polygon_ring +%type flat_coords_polygon_ring_list +%type flat_coords_polygon_ring_list_with_parens + +// MultiPoints +%type multipoint_point +%type multipoint_base_type_point +%type multipoint_non_base_type_point + +%type multipoint_base_type_point_flat_repr +%type multipoint_non_base_point_flat_repr +%type multipoint_base_type_point_list +%type multipoint_non_base_type_point_list +%type multipoint_base_type_point_list_with_parens +%type multipoint_non_base_type_point_list_with_parens + +// MultiLineStrings +%type multilinestring_base_type_linestring +%type multilinestring_non_base_type_linestring + +%type multilinestring_base_type_linestring_flat_repr +%type multilinestring_non_base_type_linestring_flat_repr +%type multilinestring_base_type_linestring_list +%type multilinestring_non_base_type_linestring_list +%type multilinestring_base_type_linestring_list_with_parens +%type multilinestring_non_base_type_linestring_list_with_parens + +// MultilPolygons +%type multipolygon_base_type_polygon +%type multipolygon_non_base_type_polygon + +%type multipolygon_base_type_polygon_multi_poly_repr +%type multipolygon_non_base_type_polygon_multi_poly_repr +%type multipolygon_base_type_polygon_list +%type multipolygon_non_base_type_polygon_list +%type multipolygon_base_type_polygon_list_with_parens +%type multipolygon_non_base_type_polygon_list_with_parens %% @@ -169,644 +186,521 @@ geometry: | multipolygon point: - POINT two_coords_point_with_parens + point_type flat_coords_point_with_parens { - $$ = geom.NewPointFlat(geom.XY, $2) + $$ = geom.NewPointFlat(wktlex.(*wktLex).curLayout, $2) } -| POINT three_coords_point_with_parens +| point_type empty { - $$ = geom.NewPointFlat(geom.XYZ, $2) + $$ = geom.NewPointEmpty(wktlex.(*wktLex).curLayout) } -| POINT four_coords_point_with_parens - { - $$ = geom.NewPointFlat(geom.XYZM, $2) - } -| POINTM three_coords_point_with_parens - { - $$ = geom.NewPointFlat(geom.XYM, $2) - } -| POINTZ three_coords_point_with_parens - { - $$ = geom.NewPointFlat(geom.XYZ, $2) - } -| POINTZM four_coords_point_with_parens - { - $$ = geom.NewPointFlat(geom.XYZM, $2) - } -| POINT EMPTY + +point_type: + POINT { - $$ = geom.NewPointEmpty(geom.XY) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } -| POINTM EMPTY +| POINTM { - $$ = geom.NewPointEmpty(geom.XYM) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } -| POINTZ EMPTY +| POINTZ { - $$ = geom.NewPointEmpty(geom.XYZ) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } -| POINTZM EMPTY +| POINTZM { - $$ = geom.NewPointEmpty(geom.XYZM) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } linestring: - LINESTRING two_coords_line - { - $$ = geom.NewLineStringFlat(geom.XY, $2) - } -| LINESTRING three_coords_line - { - $$ = geom.NewLineStringFlat(geom.XYZ, $2) - } -| LINESTRING four_coords_line - { - $$ = geom.NewLineStringFlat(geom.XYZM, $2) - } -| LINESTRINGM three_coords_line - { - $$ = geom.NewLineStringFlat(geom.XYM, $2) - } -| LINESTRINGZ three_coords_line + linestring_type flat_coords_linestring { - $$ = geom.NewLineStringFlat(geom.XYZ, $2) + $$ = geom.NewLineStringFlat(wktlex.(*wktLex).curLayout, $2) } -| LINESTRINGZM four_coords_line +| linestring_type empty { - $$ = geom.NewLineStringFlat(geom.XYZM, $2) + $$ = geom.NewLineString(wktlex.(*wktLex).curLayout) } -| LINESTRING EMPTY + +linestring_type: + LINESTRING { - $$ = geom.NewLineString(geom.XY) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } -| LINESTRINGM EMPTY +| LINESTRINGM { - $$ = geom.NewLineString(geom.XYM) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } -| LINESTRINGZ EMPTY +| LINESTRINGZ { - $$ = geom.NewLineString(geom.XYZ) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } -| LINESTRINGZM EMPTY +| LINESTRINGZM { - $$ = geom.NewLineString(geom.XYZM) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } polygon: - POLYGON two_coords_polygon - { - $$ = geom.NewPolygonFlat(geom.XY, $2.flatCoords, $2.ends) - } -| POLYGON three_coords_polygon + polygon_type flat_coords_polygon_ring_list_with_parens { - $$ = geom.NewPolygonFlat(geom.XYZ, $2.flatCoords, $2.ends) + $$ = geom.NewPolygonFlat(wktlex.(*wktLex).curLayout, $2.flatCoords, $2.ends) } -| POLYGON four_coords_polygon +| polygon_type empty { - $$ = geom.NewPolygonFlat(geom.XYZM, $2.flatCoords, $2.ends) + $$ = geom.NewPolygon(wktlex.(*wktLex).curLayout) } -| POLYGONM three_coords_polygon - { - $$ = geom.NewPolygonFlat(geom.XYM, $2.flatCoords, $2.ends) - } -| POLYGONZ three_coords_polygon - { - $$ = geom.NewPolygonFlat(geom.XYZ, $2.flatCoords, $2.ends) - } -| POLYGONZM four_coords_polygon - { - $$ = geom.NewPolygonFlat(geom.XYZM, $2.flatCoords, $2.ends) - } -| POLYGON EMPTY + +polygon_type: + POLYGON { - $$ = geom.NewPolygon(geom.XY) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } -| POLYGONM EMPTY +| POLYGONM { - $$ = geom.NewPolygon(geom.XYM) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } -| POLYGONZ EMPTY +| POLYGONZ { - $$ = geom.NewPolygon(geom.XYZ) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } -| POLYGONZM EMPTY +| POLYGONZM { - $$ = geom.NewPolygon(geom.XYZM) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } multipoint: - MULTIPOINT '(' two_coords_point_list_allowing_empty_points ')' - { - $$ = geom.NewMultiPointFlat(geom.XY, $3.flatCoords, geom.NewMultiPointFlatOptionWithEnds($3.ends)) - } -| MULTIPOINT '(' three_coords_point_list ')' - { - $$ = geom.NewMultiPointFlat(geom.XYZ, $3) - } -| MULTIPOINT '(' four_coords_point_list ')' - { - $$ = geom.NewMultiPointFlat(geom.XYZM, $3) - } -| MULTIPOINTM '(' three_coords_point_list_allowing_empty_points ')' + multipoint_base_type multipoint_base_type_point_list_with_parens { - $$ = geom.NewMultiPointFlat(geom.XYM, $3.flatCoords, geom.NewMultiPointFlatOptionWithEnds($3.ends)) + $$ = geom.NewMultiPointFlat( + wktlex.(*wktLex).curLayout, $2.flatCoords, geom.NewMultiPointFlatOptionWithEnds($2.ends), + ) } -| MULTIPOINTZ '(' three_coords_point_list_allowing_empty_points ')' +| multipoint_non_base_type multipoint_non_base_type_point_list_with_parens { - $$ = geom.NewMultiPointFlat(geom.XYZ, $3.flatCoords, geom.NewMultiPointFlatOptionWithEnds($3.ends)) + $$ = geom.NewMultiPointFlat( + wktlex.(*wktLex).curLayout, $2.flatCoords, geom.NewMultiPointFlatOptionWithEnds($2.ends), + ) } -| MULTIPOINTZM '(' four_coords_point_list_allowing_empty_points ')' +| multipoint_type empty { - $$ = geom.NewMultiPointFlat(geom.XYZM, $3.flatCoords, geom.NewMultiPointFlatOptionWithEnds($3.ends)) + $$ = geom.NewMultiPoint(wktlex.(*wktLex).curLayout) } -| MULTIPOINT EMPTY + +multipoint_type: + multipoint_base_type +| multipoint_non_base_type + +multipoint_base_type: + MULTIPOINT { - $$ = geom.NewMultiPoint(geom.XY) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } -| MULTIPOINTM EMPTY + +multipoint_non_base_type: + MULTIPOINTM { - $$ = geom.NewMultiPoint(geom.XYM) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } -| MULTIPOINTZ EMPTY +| MULTIPOINTZ { - $$ = geom.NewMultiPoint(geom.XYZ) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } -| MULTIPOINTZM EMPTY +| MULTIPOINTZM { - $$ = geom.NewMultiPoint(geom.XYZM) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } multilinestring: - MULTILINESTRING '(' two_coords_line_list_allowing_empty_lines ')' - { - $$ = geom.NewMultiLineStringFlat(geom.XY, $3.flatCoords, $3.ends) - } -| MULTILINESTRING '(' three_coords_line_list ')' - { - $$ = geom.NewMultiLineStringFlat(geom.XYZ, $3.flatCoords, $3.ends) - } -| MULTILINESTRING '(' four_coords_line_list ')' - { - $$ = geom.NewMultiLineStringFlat(geom.XYZM, $3.flatCoords, $3.ends) - } -| MULTILINESTRINGM '(' three_coords_line_list_allowing_empty_lines ')' + multilinestring_base_type multilinestring_base_type_linestring_list_with_parens { - $$ = geom.NewMultiLineStringFlat(geom.XYM, $3.flatCoords, $3.ends) + $$ = geom.NewMultiLineStringFlat(wktlex.(*wktLex).curLayout, $2.flatCoords, $2.ends) } -| MULTILINESTRINGZ '(' three_coords_line_list_allowing_empty_lines ')' +| multilinestring_non_base_type multilinestring_non_base_type_linestring_list_with_parens { - $$ = geom.NewMultiLineStringFlat(geom.XYZ, $3.flatCoords, $3.ends) + $$ = geom.NewMultiLineStringFlat(wktlex.(*wktLex).curLayout, $2.flatCoords, $2.ends) } -| MULTILINESTRINGZM '(' four_coords_line_list_allowing_empty_lines ')' +| multilinestring_type empty { - $$ = geom.NewMultiLineStringFlat(geom.XYZM, $3.flatCoords, $3.ends) + $$ = geom.NewMultiLineString(wktlex.(*wktLex).curLayout) } -| MULTILINESTRING EMPTY + +multilinestring_type: + multilinestring_base_type +| multilinestring_non_base_type + +multilinestring_base_type: + MULTILINESTRING { - $$ = geom.NewMultiLineString(geom.XY) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } -| MULTILINESTRINGM EMPTY + +multilinestring_non_base_type: + MULTILINESTRINGM { - $$ = geom.NewMultiLineString(geom.XYM) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } -| MULTILINESTRINGZ EMPTY +| MULTILINESTRINGZ { - $$ = geom.NewMultiLineString(geom.XYZ) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } -| MULTILINESTRINGZM EMPTY +| MULTILINESTRINGZM { - $$ = geom.NewMultiLineString(geom.XYZM) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } multipolygon: - MULTIPOLYGON '(' two_coords_polygon_list_allowing_empty_polygons ')' + multipolygon_base_type multipolygon_base_type_polygon_list_with_parens { - $$ = geom.NewMultiPolygonFlat(geom.XY, $3.flatCoords, $3.endss) + $$ = geom.NewMultiPolygonFlat(wktlex.(*wktLex).curLayout, $2.flatCoords, $2.endss) } -| MULTIPOLYGON '(' three_coords_polygon_list ')' +| multipolygon_non_base_type multipolygon_non_base_type_polygon_list_with_parens { - $$ = geom.NewMultiPolygonFlat(geom.XYZ, $3.flatCoords, $3.endss) + $$ = geom.NewMultiPolygonFlat(wktlex.(*wktLex).curLayout, $2.flatCoords, $2.endss) } -| MULTIPOLYGON '(' four_coords_polygon_list ')' +| multipolygon_type empty { - $$ = geom.NewMultiPolygonFlat(geom.XYZM, $3.flatCoords, $3.endss) + $$ = geom.NewMultiPolygon(wktlex.(*wktLex).curLayout) } -| MULTIPOLYGONM '(' three_coords_polygon_list_allowing_empty_polygons ')' - { - $$ = geom.NewMultiPolygonFlat(geom.XYM, $3.flatCoords, $3.endss) - } -| MULTIPOLYGONZ '(' three_coords_polygon_list_allowing_empty_polygons ')' - { - $$ = geom.NewMultiPolygonFlat(geom.XYZ, $3.flatCoords, $3.endss) - } -| MULTIPOLYGONZM '(' four_coords_polygon_list_allowing_empty_polygons ')' - { - $$ = geom.NewMultiPolygonFlat(geom.XYZM, $3.flatCoords, $3.endss) - } -| MULTIPOLYGON EMPTY - { - $$ = geom.NewMultiPolygon(geom.XY) - } -| MULTIPOLYGONM EMPTY + +multipolygon_type: + multipolygon_base_type +| multipolygon_non_base_type + +multipolygon_base_type: + MULTIPOLYGON { - $$ = geom.NewMultiPolygon(geom.XYM) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } -| MULTIPOLYGONZ EMPTY + +multipolygon_non_base_type: + MULTIPOLYGONM { - $$ = geom.NewMultiPolygon(geom.XYZ) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } -| MULTIPOLYGONZM EMPTY +| MULTIPOLYGONZ { - $$ = geom.NewMultiPolygon(geom.XYZM) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } - -three_coords_polygon_list: - three_coords_polygon_list ',' three_coords_polygon_multi_poly_flat_repr +| MULTIPOLYGONZM { - $$ = appendMultiPolygonFlatCoordsRepr($1, $3) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } -| three_coords_polygon_multi_poly_flat_repr -four_coords_polygon_list: - four_coords_polygon_list ',' four_coords_polygon_multi_poly_flat_repr +multipolygon_base_type_polygon_list_with_parens: + '(' multipolygon_base_type_polygon_list ')' { - $$ = appendMultiPolygonFlatCoordsRepr($1, $3) + $$ = $2 } -| four_coords_polygon_multi_poly_flat_repr -two_coords_polygon_list_allowing_empty_polygons: - two_coords_polygon_list_allowing_empty_polygons ',' two_coords_polygon_allowing_empty +multipolygon_non_base_type_polygon_list_with_parens: + '(' multipolygon_non_base_type_polygon_list ')' { - $$ = appendMultiPolygonFlatCoordsRepr($1, $3) + $$ = $2 } -| two_coords_polygon_allowing_empty -three_coords_polygon_list_allowing_empty_polygons: - three_coords_polygon_list_allowing_empty_polygons ',' three_coords_polygon_allowing_empty +multipolygon_non_base_type_polygon_list: + multipolygon_non_base_type_polygon_list ',' multipolygon_non_base_type_polygon_multi_poly_repr { $$ = appendMultiPolygonFlatCoordsRepr($1, $3) } -| three_coords_polygon_allowing_empty +| multipolygon_non_base_type_polygon_multi_poly_repr -four_coords_polygon_list_allowing_empty_polygons: - four_coords_polygon_list_allowing_empty_polygons ',' four_coords_polygon_allowing_empty +multipolygon_base_type_polygon_list: + multipolygon_base_type_polygon_list ',' multipolygon_base_type_polygon_multi_poly_repr { $$ = appendMultiPolygonFlatCoordsRepr($1, $3) } -| four_coords_polygon_allowing_empty - -two_coords_polygon_allowing_empty: - two_coords_polygon_multi_poly_flat_repr -| empty_polygon_multi_poly_flat_repr - -three_coords_polygon_allowing_empty: - three_coords_polygon_multi_poly_flat_repr -| empty_polygon_multi_poly_flat_repr - -four_coords_polygon_allowing_empty: - four_coords_polygon_multi_poly_flat_repr -| empty_polygon_multi_poly_flat_repr +| multipolygon_base_type_polygon_multi_poly_repr -two_coords_polygon_multi_poly_flat_repr: - two_coords_polygon +multipolygon_base_type_polygon_multi_poly_repr: + multipolygon_base_type_polygon { $$ = makeMultiPolygonFlatCoordsRepr($1) } -three_coords_polygon_multi_poly_flat_repr: - three_coords_polygon +multipolygon_non_base_type_polygon_multi_poly_repr: + multipolygon_non_base_type_polygon { $$ = makeMultiPolygonFlatCoordsRepr($1) } -four_coords_polygon_multi_poly_flat_repr: - four_coords_polygon +multipolygon_base_type_polygon: + flat_coords_polygon_ring_list_with_parens +| empty_in_base_type_collection { - $$ = makeMultiPolygonFlatCoordsRepr($1) - } - -empty_polygon_multi_poly_flat_repr: - empty_polygon - { - $$ = makeMultiPolygonFlatCoordsRepr($1) + $$ = makeGeomFlatCoordsRepr($1) } -two_coords_polygon: - '(' two_coords_ring_list ')' +multipolygon_non_base_type_polygon: + flat_coords_polygon_ring_list_with_parens +| empty { - $$ = $2 + $$ = makeGeomFlatCoordsRepr($1) } -three_coords_polygon: - '(' three_coords_ring_list ')' +multilinestring_base_type_linestring_list_with_parens: + '(' multilinestring_base_type_linestring_list ')' { $$ = $2 } -four_coords_polygon: - '(' four_coords_ring_list ')' +multilinestring_non_base_type_linestring_list_with_parens: + '(' multilinestring_non_base_type_linestring_list ')' { $$ = $2 } -empty_polygon: - EMPTY - { - $$ = makeGeomFlatCoordsRepr(nil) - } - -two_coords_ring_list: - two_coords_ring_list ',' two_coords_ring - { - $$ = appendGeomFlatCoordsReprs($1, $3) - } -| two_coords_ring - -three_coords_ring_list: - three_coords_ring_list ',' three_coords_ring +multilinestring_base_type_linestring_list: + multilinestring_base_type_linestring_list ',' multilinestring_base_type_linestring_flat_repr { $$ = appendGeomFlatCoordsReprs($1, $3) } -| three_coords_ring +| multilinestring_base_type_linestring_flat_repr -four_coords_ring_list: - four_coords_ring_list ',' four_coords_ring +multilinestring_non_base_type_linestring_list: + multilinestring_non_base_type_linestring_list ',' multilinestring_non_base_type_linestring_flat_repr { $$ = appendGeomFlatCoordsReprs($1, $3) } -| four_coords_ring - -two_coords_ring: - two_coords_list_with_parens - { - if !isValidPolygonRing(wktlex, $1, 2) { - return 1 - } - $$ = makeGeomFlatCoordsRepr($1) - } +| multilinestring_non_base_type_linestring_flat_repr -three_coords_ring: - three_coords_list_with_parens +multilinestring_base_type_linestring_flat_repr: + multilinestring_base_type_linestring { - if !isValidPolygonRing(wktlex, $1, 3) { - return 1 - } $$ = makeGeomFlatCoordsRepr($1) } -four_coords_ring: - four_coords_list_with_parens +multilinestring_non_base_type_linestring_flat_repr: + multilinestring_non_base_type_linestring { - if !isValidPolygonRing(wktlex, $1, 4) { - return 1 - } $$ = makeGeomFlatCoordsRepr($1) } -// NB: A two_coords_line_list is not required since a 2D list inside a MULTILINESTRING is always allowed to have EMPTYs. +multilinestring_base_type_linestring: + flat_coords_linestring +| empty_in_base_type_collection -three_coords_line_list: - three_coords_line_list ',' three_coords_line_flat_repr - { - $$ = appendGeomFlatCoordsReprs($1, $3) - } -| three_coords_line_flat_repr +multilinestring_non_base_type_linestring: + flat_coords_linestring +| empty -four_coords_line_list: - four_coords_line_list ',' four_coords_line_flat_repr +multipoint_base_type_point_list_with_parens: + '(' multipoint_base_type_point_list ')' { - $$ = appendGeomFlatCoordsReprs($1, $3) + $$ = $2 } -| four_coords_line_flat_repr -two_coords_line_list_allowing_empty_lines: - two_coords_line_list_allowing_empty_lines ',' two_coords_line_allowing_empty +multipoint_non_base_type_point_list_with_parens: + '(' multipoint_non_base_type_point_list ')' { - $$ = appendGeomFlatCoordsReprs($1, $3) + $$ = $2 } -| two_coords_line_allowing_empty -three_coords_line_list_allowing_empty_lines: - three_coords_line_list_allowing_empty_lines ',' three_coords_line_allowing_empty +multipoint_base_type_point_list: + multipoint_base_type_point_list ',' multipoint_base_type_point_flat_repr { $$ = appendGeomFlatCoordsReprs($1, $3) } -| three_coords_line_allowing_empty +| multipoint_base_type_point_flat_repr -four_coords_line_list_allowing_empty_lines: - four_coords_line_list_allowing_empty_lines ',' four_coords_line_allowing_empty +multipoint_non_base_type_point_list: + multipoint_non_base_type_point_list ',' multipoint_non_base_point_flat_repr { $$ = appendGeomFlatCoordsReprs($1, $3) } -| four_coords_line_allowing_empty - -two_coords_line_allowing_empty: - two_coords_line_flat_repr -| empty_line_flat_repr - -three_coords_line_allowing_empty: - three_coords_line_flat_repr -| empty_line_flat_repr +| multipoint_non_base_point_flat_repr -four_coords_line_allowing_empty: - four_coords_line_flat_repr -| empty_line_flat_repr - -two_coords_line_flat_repr: - two_coords_line +multipoint_base_type_point_flat_repr: + multipoint_base_type_point { $$ = makeGeomFlatCoordsRepr($1) } -three_coords_line_flat_repr: - three_coords_line +multipoint_non_base_point_flat_repr: + multipoint_non_base_type_point { $$ = makeGeomFlatCoordsRepr($1) } -four_coords_line_flat_repr: - four_coords_line +multipoint_base_type_point: + multipoint_point +| empty_in_base_type_collection + +multipoint_non_base_type_point: + multipoint_point +| empty + +multipoint_point: + flat_coords_point +| flat_coords_point_with_parens + +flat_coords_polygon_ring_list_with_parens: + '(' flat_coords_polygon_ring_list ')' { - $$ = makeGeomFlatCoordsRepr($1) + $$ = $2 } -two_coords_line: - two_coords_list_with_parens +flat_coords_polygon_ring_list: + flat_coords_polygon_ring_list ',' flat_coords_polygon_ring { - if !isValidLineString(wktlex, $1, 2) { - return 1 - } + $$ = appendGeomFlatCoordsReprs($1, $3) } +| flat_coords_polygon_ring -three_coords_line: - three_coords_list_with_parens +flat_coords_polygon_ring: + flat_coords_point_list_with_parens { - if !isValidLineString(wktlex, $1, 3) { + if !isValidPolygonRing(wktlex, $1, wktlex.(*wktLex).curLayout.Stride()) { return 1 } + $$ = makeGeomFlatCoordsRepr($1) } -four_coords_line: - four_coords_list_with_parens +flat_coords_linestring: + flat_coords_point_list_with_parens { - if !isValidLineString(wktlex, $1, 4) { + if !isValidLineString(wktlex, $1, wktlex.(*wktLex).curLayout.Stride()) { return 1 } } -two_coords_list_with_parens: - '(' two_coords_list ')' - { - $$ = $2 - } - -three_coords_list_with_parens: - '(' three_coords_list ')' +flat_coords_point_list_with_parens: + '(' flat_coords_point_list ')' { $$ = $2 } -four_coords_list_with_parens: - '(' four_coords_list ')' - { - $$ = $2 - } - -empty_line_flat_repr: - EMPTY - { - $$ = makeGeomFlatCoordsRepr(nil) - } - -two_coords_list: - two_coords_list ',' two_coords - { - $$ = append($1, $3...) - } -| two_coords - -three_coords_list: - three_coords_list ',' three_coords - { - $$ = append($1, $3...) - } -| three_coords - -four_coords_list: - four_coords_list ',' four_coords - { - $$ = append($1, $3...) - } -| four_coords - -// NB: A two_coords_point_list is not required since a 2D list inside a MULTIPOINT is always allowed to have EMPTYs. - -three_coords_point_list: - three_coords_point_list ',' three_coords_point - { - $$ = append($1, $3...) - } -| three_coords_point - -four_coords_point_list: - four_coords_point_list ',' four_coords_point +flat_coords_point_list: + flat_coords_point_list ',' flat_coords_point { $$ = append($1, $3...) } -| four_coords_point +| flat_coords_point -two_coords_point_list_allowing_empty_points: - two_coords_point_list_allowing_empty_points ',' two_coords_point_allowing_empty +flat_coords_point_with_parens: + '(' flat_coords_point ')' { - $$ = appendGeomFlatCoordsReprs($1, $3) + $$ = $2 } -| two_coords_point_allowing_empty -three_coords_point_list_allowing_empty_points: - three_coords_point_list_allowing_empty_points ',' three_coords_point_allowing_empty +flat_coords_point: + flat_coords { - $$ = appendGeomFlatCoordsReprs($1, $3) + switch len($1) { + case 1: + wktlex.(*wktLex).setParseError("not enough coordinates", "each point needs at least 2 coords") + return 1 + case 2, 3, 4: + ok := wktlex.(*wktLex).validateStrideAndSetLayoutIfNoLayout(len($1)) + if !ok { + return 1 + } + default: + wktlex.(*wktLex).setParseError("too many coordinates", "each point can have at most 4 coords") + return 1 + } } -| three_coords_point_allowing_empty -four_coords_point_list_allowing_empty_points: - four_coords_point_list_allowing_empty_points ',' four_coords_point_allowing_empty +flat_coords: + flat_coords NUM { - $$ = appendGeomFlatCoordsReprs($1, $3) + $$ = append($1, $2) } -| four_coords_point_allowing_empty - -two_coords_point_allowing_empty: - two_coords_point +| NUM { - $$ = makeGeomFlatCoordsRepr($1) + $$ = []float64{$1} } -| empty_point -three_coords_point_allowing_empty: - three_coords_point +empty: + flat_coords_empty { - $$ = makeGeomFlatCoordsRepr($1) + wktlex.(*wktLex).setLayoutIfNoLayout(geom.XY) } -| empty_point -four_coords_point_allowing_empty: - four_coords_point +empty_in_base_type_collection: + flat_coords_empty { - $$ = makeGeomFlatCoordsRepr($1) + ok := wktlex.(*wktLex).setLayoutEmptyInCollection() + if !ok { + return 1 + } } -| empty_point - -two_coords_point: - two_coords -| two_coords_point_with_parens - -three_coords_point: - three_coords -| three_coords_point_with_parens - -four_coords_point: - four_coords -| four_coords_point_with_parens -empty_point: +flat_coords_empty: EMPTY { - $$ = makeGeomFlatCoordsRepr(nil) - } - -two_coords_point_with_parens: - '(' two_coords ')' - { - $$ = $2 - } - -three_coords_point_with_parens: - '(' three_coords ')' - { - $$ = $2 - } - -four_coords_point_with_parens: - '(' four_coords ')' - { - $$ = $2 - } - -two_coords: - NUM NUM - { - $$ = []float64{$1, $2} - } - -three_coords: - NUM NUM NUM - { - $$ = []float64{$1, $2, $3} - } - -four_coords: - NUM NUM NUM NUM - { - $$ = []float64{$1, $2, $3, $4} + $$ = []float64(nil) } diff --git a/pkg/geo/wkt/wkt_generated.go b/pkg/geo/wkt/wkt_generated.go index ed0ac3dbc993..c8dae3ccdb6e 100644 --- a/pkg/geo/wkt/wkt_generated.go +++ b/pkg/geo/wkt/wkt_generated.go @@ -12,7 +12,7 @@ import "github.com/twpayne/go-geom" func isValidLineString(wktlex wktLexer, flatCoords []float64, stride int) bool { if len(flatCoords) < 2*stride { - wktlex.(*wktLex).Error("syntax error: non-empty linestring with only one point") + wktlex.(*wktLex).setParseError("non-empty linestring with only one point", "minimum number of points is 2") return false } return true @@ -20,12 +20,12 @@ func isValidLineString(wktlex wktLexer, flatCoords []float64, stride int) bool { func isValidPolygonRing(wktlex wktLexer, flatCoords []float64, stride int) bool { if len(flatCoords) < 4*stride { - wktlex.(*wktLex).Error("syntax error: polygon ring doesn't have enough points") + wktlex.(*wktLex).setParseError("polygon ring doesn't have enough points", "minimum number of points is 4") return false } for i := 0; i < stride; i++ { if flatCoords[i] != flatCoords[len(flatCoords)-stride+i] { - wktlex.(*wktLex).Error("syntax error: polygon ring not closed") + wktlex.(*wktLex).setParseError("polygon ring not closed", "ensure first and last point are the same") return false } } @@ -173,182 +173,113 @@ var wktExca = [...]int{ const wktPrivate = 57344 -const wktLast = 323 +const wktLast = 158 var wktAct = [...]int{ - 50, 99, 181, 184, 179, 185, 180, 162, 157, 159, - 138, 163, 147, 168, 142, 136, 123, 116, 51, 141, - 137, 115, 128, 158, 178, 156, 236, 237, 135, 235, - 234, 233, 234, 231, 232, 229, 230, 227, 228, 52, - 225, 226, 224, 223, 114, 222, 223, 220, 221, 218, - 219, 216, 217, 214, 215, 108, 213, 212, 211, 212, - 100, 209, 210, 100, 130, 207, 208, 117, 205, 206, - 203, 204, 201, 202, 199, 200, 109, 132, 189, 109, - 197, 198, 195, 196, 188, 118, 127, 177, 118, 74, - 125, 187, 143, 151, 172, 149, 124, 170, 169, 164, - 186, 246, 101, 193, 194, 74, 119, 148, 139, 177, - 101, 69, 69, 160, 119, 134, 104, 45, 110, 61, - 182, 134, 103, 40, 104, 45, 110, 9, 10, 11, - 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, - 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, - 32, 56, 134, 242, 250, 134, 102, 37, 177, 155, - 261, 61, 155, 155, 56, 246, 103, 40, 177, 155, - 66, 53, 98, 96, 97, 95, 94, 92, 93, 91, - 90, 88, 89, 87, 86, 84, 85, 83, 82, 80, - 81, 79, 78, 76, 77, 75, 241, 73, 71, 74, - 69, 117, 68, 65, 69, 66, 53, 60, 132, 61, - 58, 55, 56, 56, 49, 1, 53, 127, 44, 243, - 45, 118, 248, 249, 247, 252, 143, 151, 253, 251, - 255, 172, 258, 257, 259, 254, 164, 262, 263, 264, - 186, 265, 260, 256, 119, 245, 242, 42, 108, 40, - 39, 36, 40, 37, 267, 266, 104, 103, 240, 239, - 238, 192, 117, 244, 191, 190, 102, 174, 175, 154, - 153, 173, 133, 183, 165, 167, 166, 171, 152, 176, - 161, 144, 33, 46, 48, 64, 62, 59, 72, 63, - 67, 70, 47, 54, 57, 131, 146, 145, 150, 129, - 140, 120, 122, 121, 126, 35, 113, 112, 43, 34, - 38, 41, 111, 107, 106, 105, 8, 7, 6, 5, - 4, 3, 2, + 92, 48, 106, 84, 109, 104, 89, 85, 113, 96, + 46, 51, 55, 99, 54, 61, 94, 81, 66, 83, + 50, 71, 129, 130, 52, 78, 21, 22, 23, 24, + 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, + 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, + 127, 128, 125, 126, 53, 72, 123, 124, 121, 122, + 87, 76, 119, 120, 70, 87, 102, 97, 117, 118, + 87, 112, 107, 111, 68, 101, 115, 116, 65, 63, + 91, 79, 49, 74, 47, 49, 49, 56, 53, 49, + 60, 47, 58, 74, 114, 49, 86, 20, 19, 18, + 17, 16, 15, 14, 13, 12, 45, 11, 10, 9, + 1, 69, 67, 108, 103, 110, 105, 64, 62, 98, + 93, 100, 87, 95, 131, 59, 87, 102, 97, 134, + 87, 112, 107, 111, 137, 138, 101, 57, 133, 88, + 136, 135, 91, 79, 132, 80, 90, 82, 77, 75, + 73, 8, 7, 6, 5, 4, 3, 2, } var wktPact = [...]int{ - 123, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 223, - 222, 219, 190, 186, 183, 182, 179, 175, 174, 170, - 169, 165, 164, 161, 160, 157, 156, 153, 152, 149, - 148, 145, 144, -1000, -1000, -1000, -1000, 237, -1000, -1000, - 228, -1000, -1000, -1000, -1000, 227, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 237, -1000, -1000, 228, -1000, -1000, -1000, - -1000, 227, -1000, -1000, -1000, -1000, 176, -1000, -1000, 121, - -1000, -1000, -1000, -1000, 89, 127, -1000, 93, -1000, 93, - -1000, 87, -1000, 141, -1000, 134, -1000, 134, -1000, 131, - -1000, 140, -1000, 81, -1000, 81, -1000, 59, -1000, 60, - 53, 47, 236, 235, 232, 72, 51, 49, -1000, -1000, - -1000, 43, 41, 39, -1000, -1000, -1000, -1000, -1000, -1000, - 37, 34, 30, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 27, -1000, -1000, -1000, 25, - 22, -1000, -1000, -1000, 20, 18, 16, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 14, -1000, -1000, -1000, - 11, 9, -1000, -1000, -1000, 6, 4, 2, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 0, -1000, - -1000, -1000, -2, -5, -1000, -1000, -1000, -1000, -1000, -1000, - 231, 230, 229, -1000, 217, -1000, 228, -1000, 227, -1000, - 71, -1000, 121, -1000, 89, -1000, 124, -1000, 137, -1000, - 95, -1000, 93, -1000, -1000, 87, -1000, 135, -1000, 121, - -1000, 89, -1000, 134, -1000, -1000, 131, -1000, 130, -1000, - 82, -1000, 75, -1000, 81, -1000, -1000, 59, 226, -1000, - 226, -1000, 225, -1000, -1000, -1000, 217, -1000, -1000, -1000, - 217, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 71, -1000, -1000, -1000, -1000, -1000, -1000, + 22, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 61, + 58, 57, 62, 60, 67, 49, 48, 67, 44, 34, + 67, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 64, -1000, -1000, + -1000, -1000, -1000, 64, -1000, -1000, 24, -1000, 54, -1000, + 54, -1000, -1000, 58, -1000, 58, -1000, -1000, 57, -1000, + 57, -1000, -23, 65, -1000, 45, -1000, 37, -1000, -1000, + 31, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 27, -1000, + -1000, -1000, -1000, 25, -1000, -1000, -1000, -1000, 21, -1000, + -1000, -1000, -1000, 19, -1000, -1000, -1000, -1000, -9, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 64, -1000, 24, -1000, + 54, -1000, 54, -1000, 58, -1000, 58, -1000, 57, -1000, + 57, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, } var wktPgo = [...]int{ - 0, 322, 321, 320, 319, 318, 317, 316, 1, 22, - 64, 272, 299, 295, 315, 314, 313, 0, 18, 39, - 269, 278, 270, 44, 21, 17, 312, 307, 306, 304, - 20, 14, 303, 302, 10, 9, 16, 15, 19, 301, - 28, 300, 298, 23, 11, 12, 8, 7, 297, 296, - 281, 25, 280, 268, 271, 267, 279, 277, 6, 5, - 2, 276, 275, 13, 4, 3, 274, 24, 273, 215, + 0, 157, 156, 155, 154, 153, 152, 151, 0, 3, + 1, 150, 7, 96, 149, 24, 9, 25, 148, 2, + 19, 147, 146, 17, 6, 145, 139, 137, 125, 123, + 121, 16, 13, 120, 119, 118, 117, 116, 115, 5, + 4, 114, 113, 112, 111, 110, 109, 108, 107, 105, + 104, 103, 102, 101, 100, 99, 98, 97, } var wktR1 = [...]int{ - 0, 69, 1, 1, 1, 1, 1, 1, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 4, 4, - 4, 4, 4, 4, 4, 4, 4, 4, 5, 5, - 5, 5, 5, 5, 5, 5, 5, 5, 6, 6, - 6, 6, 6, 6, 6, 6, 6, 6, 7, 7, - 7, 7, 7, 7, 7, 7, 7, 7, 61, 61, - 62, 62, 66, 66, 67, 67, 68, 68, 63, 63, - 64, 64, 65, 65, 57, 58, 59, 60, 53, 54, - 55, 56, 26, 26, 27, 27, 28, 28, 23, 24, - 25, 48, 48, 49, 49, 50, 50, 51, 51, 52, - 52, 45, 45, 46, 46, 47, 47, 42, 43, 44, - 20, 21, 22, 17, 18, 19, 35, 14, 14, 15, - 15, 16, 16, 32, 32, 33, 33, 39, 39, 40, - 40, 41, 41, 36, 36, 37, 37, 38, 38, 29, - 29, 30, 30, 31, 31, 34, 11, 12, 13, 8, - 9, 10, + 0, 45, 1, 1, 1, 1, 1, 1, 2, 2, + 46, 46, 46, 46, 3, 3, 47, 47, 47, 47, + 4, 4, 48, 48, 48, 48, 5, 5, 5, 51, + 51, 49, 50, 50, 50, 6, 6, 6, 54, 54, + 52, 53, 53, 53, 7, 7, 7, 57, 57, 55, + 56, 56, 56, 43, 44, 42, 42, 41, 41, 39, + 40, 37, 37, 38, 38, 35, 36, 33, 33, 34, + 34, 31, 32, 29, 29, 30, 30, 27, 28, 25, + 25, 26, 26, 23, 24, 21, 21, 22, 22, 20, + 20, 19, 18, 18, 17, 16, 15, 14, 14, 13, + 12, 11, 11, 8, 9, 10, } var wktR2 = [...]int{ 0, 1, 1, 1, 1, 1, 1, 1, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, 4, 4, - 4, 4, 4, 4, 2, 2, 2, 2, 4, 4, - 4, 4, 4, 4, 2, 2, 2, 2, 4, 4, - 4, 4, 4, 4, 2, 2, 2, 2, 3, 1, - 3, 1, 3, 1, 3, 1, 3, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 3, 3, - 3, 1, 3, 1, 3, 1, 3, 1, 1, 1, - 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, - 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 3, 3, 3, 1, 3, 1, 3, - 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, + 1, 1, 1, 1, 2, 2, 1, 1, 1, 1, + 2, 2, 1, 1, 1, 1, 2, 2, 2, 1, + 1, 1, 1, 1, 1, 2, 2, 2, 1, 1, + 1, 1, 1, 1, 2, 2, 2, 1, 1, 1, + 1, 1, 1, 3, 3, 3, 1, 3, 1, 1, + 1, 1, 1, 1, 1, 3, 3, 3, 1, 3, + 1, 1, 1, 1, 1, 1, 1, 3, 3, 3, 1, 3, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 3, 3, 3, 2, - 3, 4, + 1, 3, 3, 1, 1, 1, 3, 3, 1, 3, + 1, 2, 1, 1, 1, 1, } var wktChk = [...]int{ - -1000, -69, -1, -2, -3, -4, -5, -6, -7, 4, - 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, - 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, - 25, 26, 27, -11, -12, -13, 28, 30, -12, 28, - 30, -12, 28, -13, 28, 30, -20, -21, -22, 28, - -17, -18, -19, 30, -21, 28, 30, -21, 28, -22, - 28, 30, -53, -54, -55, 28, 30, -54, 28, 30, - -54, 28, -55, 28, 30, 30, 28, 30, 28, 30, - 28, 30, 28, 30, 28, 30, 28, 30, 28, 30, - 28, 30, 28, 30, 28, 30, 28, 30, 28, -8, - -9, -10, 29, 29, 29, -14, -15, -16, -8, -9, - -10, -26, -27, -28, -23, -24, -25, -17, -18, -19, - -39, -32, -33, -36, -30, -31, -29, -34, -9, -12, - -10, -13, -8, -11, 28, -40, -37, -30, -34, -40, - -41, -38, -31, -34, -50, -48, -49, -45, -43, -44, - -42, -35, -21, -22, -20, 28, -51, -46, -43, -35, - -51, -52, -47, -44, -35, -66, -61, -62, -63, -58, - -59, -57, -60, -54, -55, -53, -56, 28, -67, -64, - -58, -60, -67, -68, -65, -59, -60, 31, 31, 31, - 29, 29, 29, 31, 32, 31, 32, 31, 32, 31, + -1000, -45, -1, -2, -3, -4, -5, -6, -7, -46, + -47, -48, -49, -50, -51, -52, -53, -54, -55, -56, + -57, 4, 5, 6, 7, 8, 9, 10, 11, 12, + 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, + 23, 24, 25, 26, 27, -13, -8, 30, -10, 28, + -16, -8, -15, 30, -19, -8, 30, -27, 30, -28, + 30, -8, -35, 30, -36, 30, -8, -43, 30, -44, + 30, -8, -12, -11, 29, -14, -12, -18, -17, -15, + -25, -23, -21, -20, -9, -12, -13, -10, -26, -24, + -22, -20, -8, -33, -31, -29, -16, -9, -34, -32, + -30, -16, -8, -41, -39, -37, -19, -9, -42, -40, + -38, -19, -8, 31, 29, 31, 32, 31, 32, 31, 32, 31, 32, 31, 32, 31, 32, 31, 32, 31, - 32, 31, 32, 31, 31, 32, 31, 32, 31, 32, - 31, 32, 31, 32, 31, 31, 32, 31, 32, 31, - 32, 31, 32, 31, 32, 31, 31, 32, 29, 29, - 29, -8, 29, -9, -10, -23, 30, -24, -25, -36, - 30, -30, -31, -37, -38, -45, -43, -44, -46, -47, - -63, 30, -58, -59, -64, -65, 29, 29, + 32, -12, -17, -23, -24, -31, -32, -39, -40, } var wktDef = [...]int{ 0, -2, 1, 2, 3, 4, 5, 6, 7, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 8, 9, 10, 14, 0, 11, 15, - 0, 12, 16, 13, 17, 0, 18, 19, 20, 24, - 120, 121, 122, 0, 21, 25, 0, 22, 26, 23, - 27, 0, 28, 29, 30, 34, 0, 31, 35, 0, - 32, 36, 33, 37, 0, 0, 44, 0, 45, 0, - 46, 0, 47, 0, 54, 0, 55, 0, 56, 0, - 57, 0, 64, 0, 65, 0, 66, 0, 67, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 128, 130, - 132, 0, 0, 0, 93, 95, 97, 98, 99, 100, - 0, 0, 0, 138, 134, 136, 143, 144, 151, 152, - 153, 154, 149, 150, 155, 0, 140, 145, 146, 0, - 0, 142, 147, 148, 0, 0, 0, 106, 102, 104, - 111, 112, 118, 119, 117, 126, 0, 108, 113, 114, - 0, 0, 110, 115, 116, 0, 0, 0, 73, 69, - 71, 78, 79, 85, 86, 84, 87, 91, 0, 75, - 80, 81, 0, 0, 77, 82, 83, 156, 157, 158, - 159, 0, 0, 123, 0, 124, 0, 125, 0, 88, - 0, 89, 0, 90, 0, 38, 0, 39, 0, 40, - 0, 41, 0, 42, 43, 0, 48, 0, 49, 0, - 50, 0, 51, 0, 52, 53, 0, 58, 0, 59, - 0, 60, 0, 61, 0, 62, 63, 0, 160, 160, - 0, 127, 0, 129, 131, 92, 0, 94, 96, 137, - 0, 133, 135, 139, 141, 105, 101, 103, 107, 109, - 72, 0, 68, 70, 74, 76, 161, 159, + 0, 0, 29, 30, 0, 38, 39, 0, 47, 48, + 0, 10, 11, 12, 13, 16, 17, 18, 19, 22, + 23, 24, 25, 31, 32, 33, 34, 40, 41, 42, + 43, 49, 50, 51, 52, 8, 9, 0, 103, 105, + 14, 15, 95, 0, 20, 21, 0, 26, 0, 27, + 0, 28, 35, 0, 36, 0, 37, 44, 0, 45, + 0, 46, 0, 100, 102, 0, 98, 0, 93, 94, + 0, 80, 83, 85, 86, 89, 90, 104, 0, 82, + 84, 87, 88, 0, 68, 71, 73, 74, 0, 70, + 72, 75, 76, 0, 58, 59, 61, 62, 0, 56, + 60, 63, 64, 99, 101, 96, 0, 91, 0, 77, + 0, 78, 0, 65, 0, 66, 0, 53, 0, 54, + 0, 97, 92, 79, 81, 67, 69, 57, 55, } var wktTok1 = [...]int{ @@ -708,696 +639,537 @@ wktdefault: case 1: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:159 +//line wkt.y:176 { wktlex.(*wktLex).ret = wktDollar[1].geom } case 8: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:173 +//line wkt.y:190 { - wktVAL.geom = geom.NewPointFlat(geom.XY, wktDollar[2].coordList) + wktVAL.geom = geom.NewPointFlat(wktlex.(*wktLex).curLayout, wktDollar[2].coordList) } case 9: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:177 +//line wkt.y:194 { - wktVAL.geom = geom.NewPointFlat(geom.XYZ, wktDollar[2].coordList) + wktVAL.geom = geom.NewPointEmpty(wktlex.(*wktLex).curLayout) } case 10: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:181 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:200 { - wktVAL.geom = geom.NewPointFlat(geom.XYZM, wktDollar[2].coordList) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } case 11: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:185 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:207 { - wktVAL.geom = geom.NewPointFlat(geom.XYM, wktDollar[2].coordList) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } case 12: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:189 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:214 { - wktVAL.geom = geom.NewPointFlat(geom.XYZ, wktDollar[2].coordList) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } case 13: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:193 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:221 { - wktVAL.geom = geom.NewPointFlat(geom.XYZM, wktDollar[2].coordList) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } case 14: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:197 +//line wkt.y:230 { - wktVAL.geom = geom.NewPointEmpty(geom.XY) + wktVAL.geom = geom.NewLineStringFlat(wktlex.(*wktLex).curLayout, wktDollar[2].coordList) } case 15: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:201 +//line wkt.y:234 { - wktVAL.geom = geom.NewPointEmpty(geom.XYM) + wktVAL.geom = geom.NewLineString(wktlex.(*wktLex).curLayout) } case 16: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:205 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:240 { - wktVAL.geom = geom.NewPointEmpty(geom.XYZ) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } case 17: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:209 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:247 { - wktVAL.geom = geom.NewPointEmpty(geom.XYZM) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } case 18: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:215 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:254 { - wktVAL.geom = geom.NewLineStringFlat(geom.XY, wktDollar[2].coordList) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } case 19: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:219 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:261 { - wktVAL.geom = geom.NewLineStringFlat(geom.XYZ, wktDollar[2].coordList) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } case 20: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:223 +//line wkt.y:270 { - wktVAL.geom = geom.NewLineStringFlat(geom.XYZM, wktDollar[2].coordList) + wktVAL.geom = geom.NewPolygonFlat(wktlex.(*wktLex).curLayout, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) } case 21: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:227 +//line wkt.y:274 { - wktVAL.geom = geom.NewLineStringFlat(geom.XYM, wktDollar[2].coordList) + wktVAL.geom = geom.NewPolygon(wktlex.(*wktLex).curLayout) } case 22: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:231 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:280 { - wktVAL.geom = geom.NewLineStringFlat(geom.XYZ, wktDollar[2].coordList) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } case 23: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:235 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:287 { - wktVAL.geom = geom.NewLineStringFlat(geom.XYZM, wktDollar[2].coordList) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } case 24: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:239 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:294 { - wktVAL.geom = geom.NewLineString(geom.XY) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } case 25: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:243 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:301 { - wktVAL.geom = geom.NewLineString(geom.XYM) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } case 26: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:247 +//line wkt.y:310 { - wktVAL.geom = geom.NewLineString(geom.XYZ) + wktVAL.geom = geom.NewMultiPointFlat( + wktlex.(*wktLex).curLayout, wktDollar[2].flatRepr.flatCoords, geom.NewMultiPointFlatOptionWithEnds(wktDollar[2].flatRepr.ends), + ) } case 27: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:251 +//line wkt.y:316 { - wktVAL.geom = geom.NewLineString(geom.XYZM) + wktVAL.geom = geom.NewMultiPointFlat( + wktlex.(*wktLex).curLayout, wktDollar[2].flatRepr.flatCoords, geom.NewMultiPointFlatOptionWithEnds(wktDollar[2].flatRepr.ends), + ) } case 28: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:257 - { - wktVAL.geom = geom.NewPolygonFlat(geom.XY, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) - } - case 29: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:261 - { - wktVAL.geom = geom.NewPolygonFlat(geom.XYZ, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) - } - case 30: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:265 +//line wkt.y:322 { - wktVAL.geom = geom.NewPolygonFlat(geom.XYZM, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) + wktVAL.geom = geom.NewMultiPoint(wktlex.(*wktLex).curLayout) } case 31: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:269 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:332 { - wktVAL.geom = geom.NewPolygonFlat(geom.XYM, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } case 32: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:273 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:341 { - wktVAL.geom = geom.NewPolygonFlat(geom.XYZ, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } case 33: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:277 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:348 { - wktVAL.geom = geom.NewPolygonFlat(geom.XYZM, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } case 34: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:281 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:355 { - wktVAL.geom = geom.NewPolygon(geom.XY) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } case 35: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:285 +//line wkt.y:364 { - wktVAL.geom = geom.NewPolygon(geom.XYM) + wktVAL.geom = geom.NewMultiLineStringFlat(wktlex.(*wktLex).curLayout, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) } case 36: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:289 +//line wkt.y:368 { - wktVAL.geom = geom.NewPolygon(geom.XYZ) + wktVAL.geom = geom.NewMultiLineStringFlat(wktlex.(*wktLex).curLayout, wktDollar[2].flatRepr.flatCoords, wktDollar[2].flatRepr.ends) } case 37: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:293 - { - wktVAL.geom = geom.NewPolygon(geom.XYZM) - } - case 38: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:299 +//line wkt.y:372 { - wktVAL.geom = geom.NewMultiPointFlat(geom.XY, wktDollar[3].flatRepr.flatCoords, geom.NewMultiPointFlatOptionWithEnds(wktDollar[3].flatRepr.ends)) - } - case 39: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:303 - { - wktVAL.geom = geom.NewMultiPointFlat(geom.XYZ, wktDollar[3].coordList) + wktVAL.geom = geom.NewMultiLineString(wktlex.(*wktLex).curLayout) } case 40: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:307 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:382 { - wktVAL.geom = geom.NewMultiPointFlat(geom.XYZM, wktDollar[3].coordList) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } case 41: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:311 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:391 { - wktVAL.geom = geom.NewMultiPointFlat(geom.XYM, wktDollar[3].flatRepr.flatCoords, geom.NewMultiPointFlatOptionWithEnds(wktDollar[3].flatRepr.ends)) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } case 42: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:315 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:398 { - wktVAL.geom = geom.NewMultiPointFlat(geom.XYZ, wktDollar[3].flatRepr.flatCoords, geom.NewMultiPointFlatOptionWithEnds(wktDollar[3].flatRepr.ends)) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } case 43: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:319 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:405 { - wktVAL.geom = geom.NewMultiPointFlat(geom.XYZM, wktDollar[3].flatRepr.flatCoords, geom.NewMultiPointFlatOptionWithEnds(wktDollar[3].flatRepr.ends)) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } case 44: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:323 +//line wkt.y:414 { - wktVAL.geom = geom.NewMultiPoint(geom.XY) + wktVAL.geom = geom.NewMultiPolygonFlat(wktlex.(*wktLex).curLayout, wktDollar[2].multiPolyFlatRepr.flatCoords, wktDollar[2].multiPolyFlatRepr.endss) } case 45: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:327 +//line wkt.y:418 { - wktVAL.geom = geom.NewMultiPoint(geom.XYM) + wktVAL.geom = geom.NewMultiPolygonFlat(wktlex.(*wktLex).curLayout, wktDollar[2].multiPolyFlatRepr.flatCoords, wktDollar[2].multiPolyFlatRepr.endss) } case 46: wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:331 +//line wkt.y:422 { - wktVAL.geom = geom.NewMultiPoint(geom.XYZ) - } - case 47: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:335 - { - wktVAL.geom = geom.NewMultiPoint(geom.XYZM) - } - case 48: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:341 - { - wktVAL.geom = geom.NewMultiLineStringFlat(geom.XY, wktDollar[3].flatRepr.flatCoords, wktDollar[3].flatRepr.ends) + wktVAL.geom = geom.NewMultiPolygon(wktlex.(*wktLex).curLayout) } case 49: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:345 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:432 { - wktVAL.geom = geom.NewMultiLineStringFlat(geom.XYZ, wktDollar[3].flatRepr.flatCoords, wktDollar[3].flatRepr.ends) + ok := wktlex.(*wktLex).setLayout(geom.NoLayout) + if !ok { + return 1 + } } case 50: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:349 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:441 { - wktVAL.geom = geom.NewMultiLineStringFlat(geom.XYZM, wktDollar[3].flatRepr.flatCoords, wktDollar[3].flatRepr.ends) + ok := wktlex.(*wktLex).setLayout(geom.XYM) + if !ok { + return 1 + } } case 51: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:353 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:448 { - wktVAL.geom = geom.NewMultiLineStringFlat(geom.XYM, wktDollar[3].flatRepr.flatCoords, wktDollar[3].flatRepr.ends) + ok := wktlex.(*wktLex).setLayout(geom.XYZ) + if !ok { + return 1 + } } case 52: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:357 + wktDollar = wktS[wktpt-1 : wktpt+1] +//line wkt.y:455 { - wktVAL.geom = geom.NewMultiLineStringFlat(geom.XYZ, wktDollar[3].flatRepr.flatCoords, wktDollar[3].flatRepr.ends) + ok := wktlex.(*wktLex).setLayout(geom.XYZM) + if !ok { + return 1 + } } case 53: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:361 - { - wktVAL.geom = geom.NewMultiLineStringFlat(geom.XYZM, wktDollar[3].flatRepr.flatCoords, wktDollar[3].flatRepr.ends) - } - case 54: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:365 - { - wktVAL.geom = geom.NewMultiLineString(geom.XY) - } - case 55: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:369 - { - wktVAL.geom = geom.NewMultiLineString(geom.XYM) - } - case 56: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:373 - { - wktVAL.geom = geom.NewMultiLineString(geom.XYZ) - } - case 57: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:377 - { - wktVAL.geom = geom.NewMultiLineString(geom.XYZM) - } - case 58: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:383 - { - wktVAL.geom = geom.NewMultiPolygonFlat(geom.XY, wktDollar[3].multiPolyFlatRepr.flatCoords, wktDollar[3].multiPolyFlatRepr.endss) - } - case 59: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:387 - { - wktVAL.geom = geom.NewMultiPolygonFlat(geom.XYZ, wktDollar[3].multiPolyFlatRepr.flatCoords, wktDollar[3].multiPolyFlatRepr.endss) - } - case 60: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:391 - { - wktVAL.geom = geom.NewMultiPolygonFlat(geom.XYZM, wktDollar[3].multiPolyFlatRepr.flatCoords, wktDollar[3].multiPolyFlatRepr.endss) - } - case 61: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:395 - { - wktVAL.geom = geom.NewMultiPolygonFlat(geom.XYM, wktDollar[3].multiPolyFlatRepr.flatCoords, wktDollar[3].multiPolyFlatRepr.endss) - } - case 62: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:399 - { - wktVAL.geom = geom.NewMultiPolygonFlat(geom.XYZ, wktDollar[3].multiPolyFlatRepr.flatCoords, wktDollar[3].multiPolyFlatRepr.endss) - } - case 63: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:403 - { - wktVAL.geom = geom.NewMultiPolygonFlat(geom.XYZM, wktDollar[3].multiPolyFlatRepr.flatCoords, wktDollar[3].multiPolyFlatRepr.endss) - } - case 64: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:407 - { - wktVAL.geom = geom.NewMultiPolygon(geom.XY) - } - case 65: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:411 - { - wktVAL.geom = geom.NewMultiPolygon(geom.XYM) - } - case 66: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:415 - { - wktVAL.geom = geom.NewMultiPolygon(geom.XYZ) - } - case 67: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:419 - { - wktVAL.geom = geom.NewMultiPolygon(geom.XYZM) - } - case 68: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:425 - { - wktVAL.multiPolyFlatRepr = appendMultiPolygonFlatCoordsRepr(wktDollar[1].multiPolyFlatRepr, wktDollar[3].multiPolyFlatRepr) - } - case 70: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:432 +//line wkt.y:464 { - wktVAL.multiPolyFlatRepr = appendMultiPolygonFlatCoordsRepr(wktDollar[1].multiPolyFlatRepr, wktDollar[3].multiPolyFlatRepr) + wktVAL.multiPolyFlatRepr = wktDollar[2].multiPolyFlatRepr } - case 72: + case 54: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:439 +//line wkt.y:470 { - wktVAL.multiPolyFlatRepr = appendMultiPolygonFlatCoordsRepr(wktDollar[1].multiPolyFlatRepr, wktDollar[3].multiPolyFlatRepr) + wktVAL.multiPolyFlatRepr = wktDollar[2].multiPolyFlatRepr } - case 74: + case 55: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:446 +//line wkt.y:476 { wktVAL.multiPolyFlatRepr = appendMultiPolygonFlatCoordsRepr(wktDollar[1].multiPolyFlatRepr, wktDollar[3].multiPolyFlatRepr) } - case 76: + case 57: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:453 +//line wkt.y:483 { wktVAL.multiPolyFlatRepr = appendMultiPolygonFlatCoordsRepr(wktDollar[1].multiPolyFlatRepr, wktDollar[3].multiPolyFlatRepr) } - case 84: + case 59: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:472 +//line wkt.y:490 { wktVAL.multiPolyFlatRepr = makeMultiPolygonFlatCoordsRepr(wktDollar[1].flatRepr) } - case 85: + case 60: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:478 +//line wkt.y:496 { wktVAL.multiPolyFlatRepr = makeMultiPolygonFlatCoordsRepr(wktDollar[1].flatRepr) } - case 86: + case 62: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:484 +//line wkt.y:503 { - wktVAL.multiPolyFlatRepr = makeMultiPolygonFlatCoordsRepr(wktDollar[1].flatRepr) + wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) } - case 87: + case 64: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:490 +//line wkt.y:510 { - wktVAL.multiPolyFlatRepr = makeMultiPolygonFlatCoordsRepr(wktDollar[1].flatRepr) - } - case 88: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:496 - { - wktVAL.flatRepr = wktDollar[2].flatRepr + wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) } - case 89: + case 65: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:502 +//line wkt.y:516 { wktVAL.flatRepr = wktDollar[2].flatRepr } - case 90: + case 66: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:508 +//line wkt.y:522 { wktVAL.flatRepr = wktDollar[2].flatRepr } - case 91: - wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:514 - { - wktVAL.flatRepr = makeGeomFlatCoordsRepr(nil) - } - case 92: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:520 - { - wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) - } - case 94: + case 67: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:527 +//line wkt.y:528 { wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) } - case 96: + case 69: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:534 +//line wkt.y:535 { wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) } - case 98: + case 71: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:541 +//line wkt.y:542 { - if !isValidPolygonRing(wktlex, wktDollar[1].coordList, 2) { - return 1 - } wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) } - case 99: + case 72: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:550 +//line wkt.y:548 { - if !isValidPolygonRing(wktlex, wktDollar[1].coordList, 3) { - return 1 - } wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) } - case 100: - wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:559 - { - if !isValidPolygonRing(wktlex, wktDollar[1].coordList, 4) { - return 1 - } - wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) - } - case 101: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:570 - { - wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) - } - case 103: + case 77: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:577 +//line wkt.y:562 { - wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) + wktVAL.flatRepr = wktDollar[2].flatRepr } - case 105: + case 78: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:584 +//line wkt.y:568 { - wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) + wktVAL.flatRepr = wktDollar[2].flatRepr } - case 107: + case 79: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:591 +//line wkt.y:574 { wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) } - case 109: + case 81: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:598 +//line wkt.y:581 { wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) } - case 117: + case 83: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:617 +//line wkt.y:588 { wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) } - case 118: + case 84: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:623 +//line wkt.y:594 { wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) } - case 119: - wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:629 + case 91: + wktDollar = wktS[wktpt-3 : wktpt+1] +//line wkt.y:612 { - wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) + wktVAL.flatRepr = wktDollar[2].flatRepr } - case 120: - wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:635 + case 92: + wktDollar = wktS[wktpt-3 : wktpt+1] +//line wkt.y:618 { - if !isValidLineString(wktlex, wktDollar[1].coordList, 2) { - return 1 - } + wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) } - case 121: + case 94: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:643 +//line wkt.y:625 { - if !isValidLineString(wktlex, wktDollar[1].coordList, 3) { + if !isValidPolygonRing(wktlex, wktDollar[1].coordList, wktlex.(*wktLex).curLayout.Stride()) { return 1 } + wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) } - case 122: + case 95: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:651 +//line wkt.y:634 { - if !isValidLineString(wktlex, wktDollar[1].coordList, 4) { + if !isValidLineString(wktlex, wktDollar[1].coordList, wktlex.(*wktLex).curLayout.Stride()) { return 1 } } - case 123: + case 96: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:659 +//line wkt.y:642 { wktVAL.coordList = wktDollar[2].coordList } - case 124: + case 97: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:665 +//line wkt.y:648 { - wktVAL.coordList = wktDollar[2].coordList + wktVAL.coordList = append(wktDollar[1].coordList, wktDollar[3].coordList...) } - case 125: + case 99: wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:671 +//line wkt.y:655 { wktVAL.coordList = wktDollar[2].coordList } - case 126: + case 100: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:677 - { - wktVAL.flatRepr = makeGeomFlatCoordsRepr(nil) - } - case 127: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:683 - { - wktVAL.coordList = append(wktDollar[1].coordList, wktDollar[3].coordList...) - } - case 129: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:690 +//line wkt.y:661 { - wktVAL.coordList = append(wktDollar[1].coordList, wktDollar[3].coordList...) - } - case 131: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:697 - { - wktVAL.coordList = append(wktDollar[1].coordList, wktDollar[3].coordList...) - } - case 133: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:706 - { - wktVAL.coordList = append(wktDollar[1].coordList, wktDollar[3].coordList...) - } - case 135: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:713 - { - wktVAL.coordList = append(wktDollar[1].coordList, wktDollar[3].coordList...) - } - case 137: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:720 - { - wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) - } - case 139: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:727 - { - wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) + switch len(wktDollar[1].coordList) { + case 1: + wktlex.(*wktLex).setParseError("not enough coordinates", "each point needs at least 2 coords") + return 1 + case 2, 3, 4: + ok := wktlex.(*wktLex).validateStrideAndSetLayoutIfNoLayout(len(wktDollar[1].coordList)) + if !ok { + return 1 + } + default: + wktlex.(*wktLex).setParseError("too many coordinates", "each point can have at most 4 coords") + return 1 + } } - case 141: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:734 + case 101: + wktDollar = wktS[wktpt-2 : wktpt+1] +//line wkt.y:679 { - wktVAL.flatRepr = appendGeomFlatCoordsReprs(wktDollar[1].flatRepr, wktDollar[3].flatRepr) + wktVAL.coordList = append(wktDollar[1].coordList, wktDollar[2].coord) } - case 143: + case 102: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:741 +//line wkt.y:683 { - wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) + wktVAL.coordList = []float64{wktDollar[1].coord} } - case 145: + case 103: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:748 +//line wkt.y:689 { - wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) + wktlex.(*wktLex).setLayoutIfNoLayout(geom.XY) } - case 147: + case 104: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:755 +//line wkt.y:695 { - wktVAL.flatRepr = makeGeomFlatCoordsRepr(wktDollar[1].coordList) + ok := wktlex.(*wktLex).setLayoutEmptyInCollection() + if !ok { + return 1 + } } - case 155: + case 105: wktDollar = wktS[wktpt-1 : wktpt+1] -//line wkt.y:774 - { - wktVAL.flatRepr = makeGeomFlatCoordsRepr(nil) - } - case 156: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:780 - { - wktVAL.coordList = wktDollar[2].coordList - } - case 157: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:786 - { - wktVAL.coordList = wktDollar[2].coordList - } - case 158: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:792 - { - wktVAL.coordList = wktDollar[2].coordList - } - case 159: - wktDollar = wktS[wktpt-2 : wktpt+1] -//line wkt.y:798 - { - wktVAL.coordList = []float64{wktDollar[1].coord, wktDollar[2].coord} - } - case 160: - wktDollar = wktS[wktpt-3 : wktpt+1] -//line wkt.y:804 - { - wktVAL.coordList = []float64{wktDollar[1].coord, wktDollar[2].coord, wktDollar[3].coord} - } - case 161: - wktDollar = wktS[wktpt-4 : wktpt+1] -//line wkt.y:810 +//line wkt.y:704 { - wktVAL.coordList = []float64{wktDollar[1].coord, wktDollar[2].coord, wktDollar[3].coord, wktDollar[4].coord} + wktVAL.coordList = []float64(nil) } } goto wktstack /* stack new state and value */ diff --git a/pkg/geo/wkt/wkt_test.go b/pkg/geo/wkt/wkt_test.go index 19f8dd2e8094..275b0fa4593a 100644 --- a/pkg/geo/wkt/wkt_test.go +++ b/pkg/geo/wkt/wkt_test.go @@ -394,9 +394,10 @@ POINT POINT { desc: "2D point with extra comma", input: "POINT(0, 0)", - expectedErrStr: `syntax error: unexpected ',', expecting NUM at pos 7 + expectedErrStr: `syntax error: not enough coordinates at pos 7 POINT(0, 0) - ^`, + ^ +HINT: each point needs at least 2 coords`, }, { desc: "2D linestring with no points", @@ -410,28 +411,31 @@ LINESTRING() input: "LINESTRING(0 0)", expectedErrStr: `syntax error: non-empty linestring with only one point at pos 14 LINESTRING(0 0) - ^`, + ^ +HINT: minimum number of points is 2`, }, { desc: "linestring with mixed dimensionality", input: "LINESTRING(0 0, 1 1 1)", - expectedErrStr: `syntax error: unexpected NUM, expecting ')' or ',' at pos 20 + expectedErrStr: `syntax error: mixed dimensionality, parsed layout is XY so expecting 2 coords but got 3 coords at pos 21 LINESTRING(0 0, 1 1 1) - ^`, + ^`, }, { desc: "2D polygon with not enough points", input: "POLYGON((0 0, 1 1, 2 0))", expectedErrStr: `syntax error: polygon ring doesn't have enough points at pos 22 POLYGON((0 0, 1 1, 2 0)) - ^`, + ^ +HINT: minimum number of points is 4`, }, { desc: "2D polygon with ring that isn't closed", input: "POLYGON((0 0, 1 1, 2 0, 1 -1))", expectedErrStr: `syntax error: polygon ring not closed at pos 28 POLYGON((0 0, 1 1, 2 0, 1 -1)) - ^`, + ^ +HINT: ensure first and last point are the same`, }, { desc: "2D polygon with empty second ring", @@ -452,7 +456,8 @@ POLYGON((0 0, 1 -1, 2 0, 0 0), EMPTY) input: "POLYGON((0 0, 1 -1, 2 0, 0 0), (0.5 -0.5))", expectedErrStr: `syntax error: polygon ring doesn't have enough points at pos 40 POLYGON((0 0, 1 -1, 2 0, 0 0), (0.5 -0.5)) - ^`, + ^ +HINT: minimum number of points is 4`, }, { desc: "2D multipoint without any points", @@ -464,9 +469,10 @@ MULTIPOINT() { desc: "3D multipoint without comma separating points", input: "MULTIPOINT Z (0 0 0 0 0 0)", - expectedErrStr: `syntax error: unexpected NUM, expecting ')' or ',' at pos 20 + expectedErrStr: `syntax error: too many coordinates at pos 25 MULTIPOINT Z (0 0 0 0 0 0) - ^`, + ^ +HINT: each point can have at most 4 coords`, }, { desc: "2D multipoint with EMPTY inside extraneous parentheses", @@ -478,14 +484,15 @@ MULTIPOINT((EMPTY)) { desc: "3D multipoint using EMPTY as a point without using Z in type", input: "MULTIPOINT(0 0 0, EMPTY)", - expectedErrStr: `syntax error: unexpected EMPTY, expecting NUM or '(' at pos 18 + expectedErrStr: `syntax error: mixed dimensionality, parsed layout is XYZ but encountered layout of XY at pos 18 MULTIPOINT(0 0 0, EMPTY) - ^`, + ^ +HINT: EMPTY is XY layout in base geometry type collection`, }, { desc: "multipoint with mixed dimensionality", input: "MULTIPOINT(0 0 0, 1 1)", - expectedErrStr: `syntax error: unexpected ')', expecting NUM at pos 21 + expectedErrStr: `syntax error: mixed dimensionality, parsed layout is XYZ so expecting 3 coords but got 2 coords at pos 21 MULTIPOINT(0 0 0, 1 1) ^`, }, @@ -501,14 +508,15 @@ MULTILINESTRING(()) input: "MULTILINESTRING((0 0))", expectedErrStr: `syntax error: non-empty linestring with only one point at pos 20 MULTILINESTRING((0 0)) - ^`, + ^ +HINT: minimum number of points is 2`, }, { desc: "4D multilinestring using EMPTY without using ZM in type", input: "MULTILINESTRING(EMPTY, (0 0 0 0, 2 3 -2 -3))", - expectedErrStr: `syntax error: unexpected NUM, expecting ')' or ',' at pos 28 + expectedErrStr: `syntax error: mixed dimensionality, parsed layout is XY so expecting 2 coords but got 4 coords at pos 31 MULTILINESTRING(EMPTY, (0 0 0 0, 2 3 -2 -3)) - ^`, + ^`, }, { desc: "2D multipolygon with no polygons", @@ -527,23 +535,25 @@ MULTIPOLYGON((1 0, 2 5, -2 5, 1 0)) { desc: "multipolygon with mixed dimensionality", input: "MULTIPOLYGON(((1 0, 2 5, -2 5, 1 0)), ((1 0 2, 2 5 1, -2 5 -1, 1 0 2)))", - expectedErrStr: `syntax error: unexpected NUM, expecting ')' or ',' at pos 44 + expectedErrStr: `syntax error: mixed dimensionality, parsed layout is XY so expecting 2 coords but got 3 coords at pos 45 MULTIPOLYGON(((1 0, 2 5, -2 5, 1 0)), ((1 0 2, 2 5 1, -2 5 -1, 1 0 2))) - ^`, + ^`, }, { desc: "2D multipolygon with polygon that doesn't have enough points", input: "MULTIPOLYGON(((0 0, 1 1, 2 0)))", expectedErrStr: `syntax error: polygon ring doesn't have enough points at pos 28 MULTIPOLYGON(((0 0, 1 1, 2 0))) - ^`, + ^ +HINT: minimum number of points is 4`, }, { desc: "2D multipolygon with polygon with ring that isn't closed", input: "MULTIPOLYGON(((0 0, 1 1, 2 0, 1 -1)))", expectedErrStr: `syntax error: polygon ring not closed at pos 34 MULTIPOLYGON(((0 0, 1 1, 2 0, 1 -1))) - ^`, + ^ +HINT: ensure first and last point are the same`, }, { desc: "2D multipolygon with polygon with empty second ring", @@ -564,14 +574,15 @@ MULTIPOLYGON(((0 0, 1 -1, 2 0, 0 0), EMPTY)) input: "MULTIPOLYGON(((0 0, 1 -1, 2 0, 0 0), (0.5 -0.5)))", expectedErrStr: `syntax error: polygon ring doesn't have enough points at pos 46 MULTIPOLYGON(((0 0, 1 -1, 2 0, 0 0), (0.5 -0.5))) - ^`, + ^ +HINT: minimum number of points is 4`, }, { desc: "3D multipolygon using EMPTY without using Z in its type", input: "MULTIPOLYGON(EMPTY, ((0 0 0, 1 1 1, 2 3 1, 0 0 0)))", - expectedErrStr: `syntax error: unexpected NUM, expecting ')' or ',' at pos 26 + expectedErrStr: `syntax error: mixed dimensionality, parsed layout is XY so expecting 2 coords but got 3 coords at pos 27 MULTIPOLYGON(EMPTY, ((0 0 0, 1 1 1, 2 3 1, 0 0 0))) - ^`, + ^`, }, }