house numbers.
```sql
-FUNCTION token_addr_street_match_tokens(info JSONB) RETURNS INTEGER[]
+FUNCTION token_matches_street(info JSONB, street_tokens INTEGER[]) RETURNS BOOLEAN
```
-Return the match token IDs by which to search a matching street from the
-`addr:street` tag. These IDs will be matched against the IDs supplied by
-`token_get_name_match_tokens`. Must be NULL when the place has no `addr:street`
-tag.
+Check if the given tokens (previously saved from `token_get_name_match_tokens()`)
+match against the `addr:street` tag name. Must return either NULL or FALSE
+when the place has no `addr:street` tag.
```sql
-FUNCTION token_addr_place_match_tokens(info JSONB) RETURNS INTEGER[]
+FUNCTION token_matches_place(info JSONB, place_tokens INTEGER[]) RETURNS BOOLEAN
```
-Return the match token IDs by which to search a matching place from the
-`addr:place` tag. These IDs will be matched against the IDs supplied by
-`token_get_name_match_tokens`. Must be NULL when the place has no `addr:place`
-tag.
+Check if the given tokens (previously saved from `token_get_name_match_tokens()`)
+match against the `addr:place` tag name. Must return either NULL or FALSE
+when the place has no `addr:place` tag.
+
```sql
FUNCTION token_addr_place_search_tokens(info JSONB) RETURNS INTEGER[]
database. Must be NULL when the place has no `addr:place` tag.
```sql
-CREATE TYPE token_addresstoken AS (
- key TEXT,
- match_tokens INT[],
- search_tokens INT[]
-);
+FUNCTION token_get_address_keys(info JSONB) RETURNS SETOF TEXT
+```
+
+Return the set of keys for which address information is provided. This
+should correspond to the list of (relevant) `addr:*` tags with the `addr:`
+prefix removed or the keys used in the `address` dictionary of the place info.
-FUNCTION token_get_address_tokens(info JSONB) RETURNS SETOF token_addresstoken
+```sql
+FUNCTION token_get_address_search_tokens(info JSONB, key TEXT) RETURNS INTEGER[]
```
-Return the match and search token IDs for explicit `addr:*` tags for the place
-other than `addr:street` and `addr:place`. For each address item there are
-three pieces of information returned:
-
- * _key_ contains the type of address item (city, county, etc.). This is the
- key handed in with the `address` dictionary.
- * *match_tokens* is the list of token IDs used to find the corresponding
- place object for the address part. The list is matched against the IDs
- from `token_get_name_match_tokens`.
- * *search_tokens* is the list of token IDs under which to search the address
- item. It is used when no corresponding place object was found.
+Return the array of search tokens for the given address part. `key` can be
+expected to be one of those returned with `token_get_address_keys()`. The
+search tokens are added to the address search vector of the place, when no
+corresponding OSM object could be found for the given address part from which
+to copy the name information.
+
+```sql
+FUNCTION token_matches_address(info JSONB, key TEXT, tokens INTEGER[])
+```
+
+Check if the given tokens match against the address part `key`.
+
+__Warning:__ the tokens that are handed in are the lists previously saved
+from `token_get_name_search_tokens()`, _not_ from the match token list. This
+is an historical oddity which will be fixed at some point in the future.
+Currently, tokenizers are encouraged to make sure that matching works against
+both the search token list and the match token list.
```sql
FUNCTION token_normalized_postcode(postcode TEXT) RETURNS TEXT
-- find the parent road of the cut road parts
-CREATE OR REPLACE FUNCTION get_interpolation_parent(street INTEGER[], place INTEGER[],
+CREATE OR REPLACE FUNCTION get_interpolation_parent(token_info JSONB,
partition SMALLINT,
centroid GEOMETRY, geom GEOMETRY)
RETURNS BIGINT
parent_place_id BIGINT;
location RECORD;
BEGIN
- parent_place_id := find_parent_for_address(street, place, partition, centroid);
+ parent_place_id := find_parent_for_address(token_info, partition, centroid);
IF parent_place_id is null THEN
FOR location IN SELECT place_id FROM placex
NEW.interpolationtype = NEW.address->'interpolation';
place_centroid := ST_PointOnSurface(NEW.linegeo);
- NEW.parent_place_id = get_interpolation_parent(token_addr_street_match_tokens(NEW.token_info),
- token_addr_place_match_tokens(NEW.token_info),
- NEW.partition, place_centroid, NEW.linegeo);
+ NEW.parent_place_id = get_interpolation_parent(NEW.token_info, NEW.partition,
+ place_centroid, NEW.linegeo);
interpol_postcode := token_normalized_postcode(NEW.address->'postcode');
CREATE OR REPLACE FUNCTION get_address_place(in_partition SMALLINT, feature GEOMETRY,
from_rank SMALLINT, to_rank SMALLINT,
- extent FLOAT, tokens INT[])
+ extent FLOAT, token_info JSONB, key TEXT)
RETURNS nearfeaturecentr
AS $$
DECLARE
FROM location_area_large_{{ partition }}
WHERE geometry && ST_Expand(feature, extent)
AND rank_address between from_rank and to_rank
- AND tokens && keywords
+ AND token_matches_address(token_info, key, keywords)
GROUP BY place_id, keywords, rank_address, rank_search, isguess, postcode, centroid
ORDER BY bool_or(ST_Intersects(geometry, feature)), distance LIMIT 1;
RETURN r;
CREATE OR REPLACE FUNCTION getNearestNamedRoadPlaceId(in_partition INTEGER,
point GEOMETRY,
- isin_token INTEGER[])
+ token_info JSONB)
RETURNS BIGINT
AS $$
DECLARE
parent BIGINT;
BEGIN
+ IF not token_has_addr_street(token_info) THEN
+ RETURN NULL;
+ END IF;
{% for partition in db.partitions %}
IF in_partition = {{ partition }} THEN
SELECT place_id FROM search_name_{{ partition }}
INTO parent
- WHERE name_vector && isin_token
+ WHERE token_matches_street(token_info, name_vector)
AND centroid && ST_Expand(point, 0.015)
AND address_rank between 26 and 27
ORDER BY ST_Distance(centroid, point) ASC limit 1;
CREATE OR REPLACE FUNCTION getNearestNamedPlacePlaceId(in_partition INTEGER,
point GEOMETRY,
- isin_token INTEGER[])
+ token_info JSONB)
RETURNS BIGINT
AS $$
DECLARE
parent BIGINT;
BEGIN
+ IF not token_has_addr_place(token_info) THEN
+ RETURN NULL;
+ END IF;
{% for partition in db.partitions %}
IF in_partition = {{ partition }} THEN
SELECT place_id
INTO parent
FROM search_name_{{ partition }}
- WHERE name_vector && isin_token
+ WHERE token_matches_place(token_info, name_vector)
AND centroid && ST_Expand(point, 0.04)
AND address_rank between 16 and 25
ORDER BY ST_Distance(centroid, point) ASC limit 1;
poi_osm_id BIGINT,
poi_partition SMALLINT,
bbox GEOMETRY,
- addr_street INTEGER[],
- addr_place INTEGER[],
+ token_info JSONB,
is_place_addr BOOLEAN)
RETURNS BIGINT
AS $$
parent_place_id := find_associated_street(poi_osm_type, poi_osm_id);
IF parent_place_id is null THEN
- parent_place_id := find_parent_for_address(addr_street, addr_place,
- poi_partition, bbox);
+ parent_place_id := find_parent_for_address(token_info, poi_partition, bbox);
END IF;
IF parent_place_id is null and poi_osm_type = 'N' THEN
WHERE s.place_id = parent_place_id;
FOR addr_item IN
- SELECT (get_addr_tag_rank(key, country)).*, match_tokens, search_tokens
- FROM token_get_address_tokens(token_info)
- WHERE not search_tokens <@ parent_address_vector
+ SELECT (get_addr_tag_rank(key, country)).*, key,
+ token_get_address_search_tokens(token_info, key) as search_tokens
+ FROM token_get_address_keys(token_info) as key
+ WHERE not token_get_address_search_tokens(token_info, key) <@ parent_address_vector
LOOP
addr_place := get_address_place(in_partition, geometry,
addr_item.from_rank, addr_item.to_rank,
- addr_item.extent, addr_item.match_tokens);
+ addr_item.extent, token_info, addr_item.key);
IF addr_place is null THEN
-- No place found in OSM that matches. Make it at least searchable.
FOR location IN
SELECT (get_address_place(partition, geometry, from_rank, to_rank,
- extent, match_tokens)).*, search_tokens
- FROM (SELECT (get_addr_tag_rank(key, country)).*, match_tokens, search_tokens
- FROM token_get_address_tokens(token_info)) x
+ extent, token_info, key)).*, key
+ FROM (SELECT (get_addr_tag_rank(key, country)).*, key
+ FROM token_get_address_keys(token_info) as key) x
ORDER BY rank_address, distance, isguess desc
LOOP
IF location.place_id is null THEN
{% if not db.reverse_only %}
- nameaddress_vector := array_merge(nameaddress_vector, location.search_tokens);
+ nameaddress_vector := array_merge(nameaddress_vector,
+ token_get_address_search_tokens(token_info,
+ location.key));
{% endif %}
ELSE
{% if not db.reverse_only %}
parent_address_level SMALLINT;
place_address_level SMALLINT;
- addr_street INTEGER[];
- addr_place INTEGER[];
-
max_rank SMALLINT;
name_vector INTEGER[];
END IF;
NEW.housenumber := token_normalized_housenumber(NEW.token_info);
- addr_street := token_addr_street_match_tokens(NEW.token_info);
- addr_place := token_addr_place_match_tokens(NEW.token_info);
NEW.postcode := null;
NEW.parent_place_id := find_parent_for_poi(NEW.osm_type, NEW.osm_id,
NEW.partition,
ST_Envelope(NEW.geometry),
- addr_street, addr_place,
+ NEW.token_info,
is_place_address);
-- If we found the road take a shortcut here.
-- Find the parent of an address with addr:street/addr:place tag.
--
--- \param street Value of addr:street or NULL if tag is missing.
--- \param place Value of addr:place or NULL if tag is missing.
+-- \param token_info Naming info with the address information.
-- \param partition Partition where to search the parent.
-- \param centroid Location of the address.
--
-- \return Place ID of the parent if one was found, NULL otherwise.
-CREATE OR REPLACE FUNCTION find_parent_for_address(street INTEGER[], place INTEGER[],
+CREATE OR REPLACE FUNCTION find_parent_for_address(token_info JSONB,
partition SMALLINT,
centroid GEOMETRY)
RETURNS BIGINT
DECLARE
parent_place_id BIGINT;
BEGIN
- IF street is not null THEN
- -- Check for addr:street attributes
- -- Note that addr:street links can only be indexed, once the street itself is indexed
- parent_place_id := getNearestNamedRoadPlaceId(partition, centroid, street);
- IF parent_place_id is not null THEN
- {% if debug %}RAISE WARNING 'Get parent form addr:street: %', parent_place_id;{% endif %}
- RETURN parent_place_id;
- END IF;
+ -- Check for addr:street attributes
+ parent_place_id := getNearestNamedRoadPlaceId(partition, centroid, token_info);
+ IF parent_place_id is not null THEN
+ {% if debug %}RAISE WARNING 'Get parent from addr:street: %', parent_place_id;{% endif %}
+ RETURN parent_place_id;
END IF;
-- Check for addr:place attributes.
- IF place is not null THEN
- parent_place_id := getNearestNamedPlacePlaceId(partition, centroid, place);
- IF parent_place_id is not null THEN
- {% if debug %}RAISE WARNING 'Get parent form addr:place: %', parent_place_id;{% endif %}
- RETURN parent_place_id;
- END IF;
- END IF;
-
- RETURN NULL;
+ parent_place_id := getNearestNamedPlacePlaceId(partition, centroid, token_info);
+ {% if debug %}RAISE WARNING 'Get parent from addr:place: %', parent_place_id;{% endif %}
+ RETURN parent_place_id;
END;
$$
LANGUAGE plpgsql STABLE;
+
CREATE OR REPLACE FUNCTION delete_location(OLD_place_id BIGINT)
RETURNS BOOLEAN
AS $$
out_partition INTEGER;
out_parent_place_id BIGINT;
location RECORD;
- address_street_word_ids INTEGER[];
BEGIN
place_centroid := ST_Centroid(linegeo);
out_partition := get_partition('us');
- out_parent_place_id := null;
- address_street_word_ids := token_addr_street_match_tokens(token_info);
- IF address_street_word_ids IS NOT NULL THEN
- out_parent_place_id := getNearestNamedRoadPlaceId(out_partition, place_centroid,
- address_street_word_ids);
- END IF;
+ out_parent_place_id := getNearestNamedRoadPlaceId(out_partition, place_centroid,
+ token_info);
IF out_parent_place_id IS NULL THEN
SELECT getNearestParallelRoadFeature(out_partition, linegeo)
$$ LANGUAGE SQL IMMUTABLE STRICT;
-CREATE OR REPLACE FUNCTION token_addr_street_match_tokens(info JSONB)
- RETURNS INTEGER[]
+CREATE OR REPLACE FUNCTION token_has_addr_street(info JSONB)
+ RETURNS BOOLEAN
+AS $$
+ SELECT info->>'street' is not null;
+$$ LANGUAGE SQL IMMUTABLE;
+
+
+CREATE OR REPLACE FUNCTION token_has_addr_place(info JSONB)
+ RETURNS BOOLEAN
AS $$
- SELECT (info->>'street')::INTEGER[]
+ SELECT info->>'place' is not null;
+$$ LANGUAGE SQL IMMUTABLE;
+
+
+CREATE OR REPLACE FUNCTION token_matches_street(info JSONB, street_tokens INTEGER[])
+ RETURNS BOOLEAN
+AS $$
+ SELECT (info->>'street')::INTEGER[] <@ street_tokens
$$ LANGUAGE SQL IMMUTABLE STRICT;
-CREATE OR REPLACE FUNCTION token_addr_place_match_tokens(info JSONB)
- RETURNS INTEGER[]
+CREATE OR REPLACE FUNCTION token_matches_place(info JSONB, place_tokens INTEGER[])
+ RETURNS BOOLEAN
AS $$
- SELECT (info->>'place_match')::INTEGER[]
+ SELECT (info->>'place')::INTEGER[] <@ place_tokens
$$ LANGUAGE SQL IMMUTABLE STRICT;
CREATE OR REPLACE FUNCTION token_addr_place_search_tokens(info JSONB)
RETURNS INTEGER[]
AS $$
- SELECT (info->>'place_search')::INTEGER[]
+ SELECT (info->>'place')::INTEGER[]
$$ LANGUAGE SQL IMMUTABLE STRICT;
-DROP TYPE IF EXISTS token_addresstoken CASCADE;
-CREATE TYPE token_addresstoken AS (
- key TEXT,
- match_tokens INT[],
- search_tokens INT[]
-);
+CREATE OR REPLACE FUNCTION token_get_address_keys(info JSONB)
+ RETURNS SETOF TEXT
+AS $$
+ SELECT * FROM jsonb_object_keys(info->'addr');
+$$ LANGUAGE SQL IMMUTABLE STRICT;
-CREATE OR REPLACE FUNCTION token_get_address_tokens(info JSONB)
- RETURNS SETOF token_addresstoken
+
+CREATE OR REPLACE FUNCTION token_get_address_search_tokens(info JSONB, key TEXT)
+ RETURNS INTEGER[]
AS $$
- SELECT key, (value->>1)::int[] as match_tokens,
- (value->>0)::int[] as search_tokens
- FROM jsonb_each(info->'addr');
+ SELECT (info->'addr'->>key)::INTEGER[];
+$$ LANGUAGE SQL IMMUTABLE STRICT;
+
+
+CREATE OR REPLACE FUNCTION token_matches_address(info JSONB, key TEXT, tokens INTEGER[])
+ RETURNS BOOLEAN
+AS $$
+ SELECT (info->'addr'->>key)::INTEGER[] <@ tokens;
$$ LANGUAGE SQL IMMUTABLE STRICT;
VALUES (term_id, term, 'w', json_build_object('count', term_count));
END IF;
- IF term_count < {{ max_word_freq }} THEN
- partial_tokens := array_merge(partial_tokens, ARRAY[term_id]);
- END IF;
+ partial_tokens := array_merge(partial_tokens, ARRAY[term_id]);
END LOOP;
END;
$$
LANGUAGE plpgsql;
+CREATE OR REPLACE FUNCTION getorcreate_partial_word(partial TEXT)
+ RETURNS INTEGER
+ AS $$
+DECLARE
+ token INTEGER;
+BEGIN
+ SELECT min(word_id) INTO token
+ FROM word WHERE word_token = partial and type = 'w';
+
+ IF token IS NULL THEN
+ token := nextval('seq_word');
+ INSERT INTO word (word_id, word_token, type, info)
+ VALUES (token, partial, 'w', json_build_object('count', 0));
+ END IF;
+
+ RETURN token;
+END;
+$$
+LANGUAGE plpgsql;
+
+
CREATE OR REPLACE FUNCTION getorcreate_hnr_id(lookup_term TEXT)
RETURNS INTEGER
AS $$
$$ LANGUAGE SQL IMMUTABLE STRICT;
-CREATE OR REPLACE FUNCTION token_addr_street_match_tokens(info JSONB)
- RETURNS INTEGER[]
+CREATE OR REPLACE FUNCTION token_has_addr_street(info JSONB)
+ RETURNS BOOLEAN
+AS $$
+ SELECT info->>'street' is not null;
+$$ LANGUAGE SQL IMMUTABLE;
+
+
+CREATE OR REPLACE FUNCTION token_has_addr_place(info JSONB)
+ RETURNS BOOLEAN
AS $$
- SELECT (info->>'street')::INTEGER[]
+ SELECT info->>'place_match' is not null;
+$$ LANGUAGE SQL IMMUTABLE;
+
+
+CREATE OR REPLACE FUNCTION token_matches_street(info JSONB, street_tokens INTEGER[])
+ RETURNS BOOLEAN
+AS $$
+ SELECT (info->>'street')::INTEGER[] && street_tokens
$$ LANGUAGE SQL IMMUTABLE STRICT;
-CREATE OR REPLACE FUNCTION token_addr_place_match_tokens(info JSONB)
- RETURNS INTEGER[]
+CREATE OR REPLACE FUNCTION token_matches_place(info JSONB, place_tokens INTEGER[])
+ RETURNS BOOLEAN
AS $$
- SELECT (info->>'place_match')::INTEGER[]
+ SELECT (info->>'place_match')::INTEGER[] && place_tokens
$$ LANGUAGE SQL IMMUTABLE STRICT;
$$ LANGUAGE SQL IMMUTABLE STRICT;
-DROP TYPE IF EXISTS token_addresstoken CASCADE;
-CREATE TYPE token_addresstoken AS (
- key TEXT,
- match_tokens INT[],
- search_tokens INT[]
-);
+CREATE OR REPLACE FUNCTION token_get_address_keys(info JSONB)
+ RETURNS SETOF TEXT
+AS $$
+ SELECT * FROM jsonb_object_keys(info->'addr');
+$$ LANGUAGE SQL IMMUTABLE STRICT;
+
+
+CREATE OR REPLACE FUNCTION token_get_address_search_tokens(info JSONB, key TEXT)
+ RETURNS INTEGER[]
+AS $$
+ SELECT (info->'addr'->key->>0)::INTEGER[];
+$$ LANGUAGE SQL IMMUTABLE STRICT;
-CREATE OR REPLACE FUNCTION token_get_address_tokens(info JSONB)
- RETURNS SETOF token_addresstoken
+
+CREATE OR REPLACE FUNCTION token_matches_address(info JSONB, key TEXT, tokens INTEGER[])
+ RETURNS BOOLEAN
AS $$
- SELECT key, (value->>1)::int[] as match_tokens,
- (value->>0)::int[] as search_tokens
- FROM jsonb_each(info->'addr');
+ SELECT (info->'addr'->key->>1)::INTEGER[] && tokens;
$$ LANGUAGE SQL IMMUTABLE STRICT;
from nominatim.tokenizer.icu_name_processor import ICUNameProcessor, ICUNameProcessorRules
from nominatim.tokenizer.base import AbstractAnalyzer, AbstractTokenizer
-DBCFG_MAXWORDFREQ = "tokenizer_maxwordfreq"
DBCFG_TERM_NORMALIZATION = "tokenizer_term_normalization"
LOG = logging.getLogger()
self.data_dir = data_dir
self.naming_rules = None
self.term_normalization = None
- self.max_word_frequency = None
def init_new_db(self, config, init_db=True):
config='TOKENIZER_CONFIG'))
self.naming_rules = ICUNameProcessorRules(loader=loader)
self.term_normalization = config.TERM_NORMALIZATION
- self.max_word_frequency = config.MAX_WORD_FREQUENCY
self._install_php(config.lib_dir.php)
- self._save_config(config)
+ self._save_config()
if init_db:
self.update_sql_functions(config)
with connect(self.dsn) as conn:
self.naming_rules = ICUNameProcessorRules(conn=conn)
self.term_normalization = get_property(conn, DBCFG_TERM_NORMALIZATION)
- self.max_word_frequency = get_property(conn, DBCFG_MAXWORDFREQ)
def finalize_import(self, _):
""" Reimport the SQL functions for this tokenizer.
"""
with connect(self.dsn) as conn:
- max_word_freq = get_property(conn, DBCFG_MAXWORDFREQ)
sqlp = SQLPreprocessor(conn, config)
- sqlp.run_sql_file(conn, 'tokenizer/icu_tokenizer.sql',
- max_word_freq=max_word_freq)
+ sqlp.run_sql_file(conn, 'tokenizer/icu_tokenizer.sql')
def check_database(self):
php_file = self.data_dir / "tokenizer.php"
php_file.write_text(dedent(f"""\
<?php
- @define('CONST_Max_Word_Frequency', {self.max_word_frequency});
+ @define('CONST_Max_Word_Frequency', 10000000);
@define('CONST_Term_Normalization_Rules', "{self.term_normalization}");
@define('CONST_Transliteration', "{self.naming_rules.search_rules}");
require_once('{phpdir}/tokenizer/icu_tokenizer.php');"""))
- def _save_config(self, config):
+ def _save_config(self):
""" Save the configuration that needs to remain stable for the given
database as database properties.
"""
with connect(self.dsn) as conn:
self.naming_rules.save_rules(conn)
- set_property(conn, DBCFG_MAXWORDFREQ, config.MAX_WORD_FREQUENCY)
set_property(conn, DBCFG_TERM_NORMALIZATION, self.term_normalization)
elif key in ('housenumber', 'streetnumber', 'conscriptionnumber'):
hnrs.append(value)
elif key == 'street':
- token_info.add_street(*self._compute_name_tokens({'name': value}))
+ token_info.add_street(self._compute_partial_tokens(value))
elif key == 'place':
- token_info.add_place(*self._compute_name_tokens({'name': value}))
+ token_info.add_place(self._compute_partial_tokens(value))
elif not key.startswith('_') and \
key not in ('country', 'full'):
- addr_terms.append((key, *self._compute_name_tokens({'name': value})))
+ addr_terms.append((key, self._compute_partial_tokens(value)))
if hnrs:
hnrs = self._split_housenumbers(hnrs)
if addr_terms:
token_info.add_address_terms(addr_terms)
+ def _compute_partial_tokens(self, name):
+ """ Normalize the given term, split it into partial words and return
+ then token list for them.
+ """
+ norm_name = self.name_processor.get_search_normalized(name)
+
+ tokens = []
+ need_lookup = []
+ for partial in norm_name.split():
+ token = self._cache.partials.get(partial)
+ if token:
+ tokens.append(token)
+ else:
+ need_lookup.append(partial)
+
+ if need_lookup:
+ with self.conn.cursor() as cur:
+ cur.execute("""SELECT word, getorcreate_partial_word(word)
+ FROM unnest(%s) word""",
+ (need_lookup, ))
+
+ for partial, token in cur:
+ tokens.append(token)
+ self._cache.partials[partial] = token
+
+ return tokens
def _compute_name_tokens(self, names):
""" Computes the full name and partial name tokens for the given
self.data['hnr'] = ';'.join(hnrs)
- def add_street(self, fulls, _):
+ def add_street(self, tokens):
""" Add addr:street match terms.
"""
- if fulls:
- self.data['street'] = self._mk_array(fulls)
+ if tokens:
+ self.data['street'] = self._mk_array(tokens)
- def add_place(self, fulls, partials):
+ def add_place(self, tokens):
""" Add addr:place search and match terms.
"""
- if fulls:
- self.data['place_search'] = self._mk_array(itertools.chain(fulls, partials))
- self.data['place_match'] = self._mk_array(fulls)
+ if tokens:
+ self.data['place'] = self._mk_array(tokens)
def add_address_terms(self, terms):
""" Add additional address terms.
"""
- tokens = {}
-
- for key, fulls, partials in terms:
- if fulls:
- tokens[key] = [self._mk_array(itertools.chain(fulls, partials)),
- self._mk_array(fulls)]
+ tokens = {key: self._mk_array(partials)
+ for key, partials in terms if partials}
if tokens:
self.data['addr'] = tokens
"""
def __init__(self):
self.names = {}
+ self.partials = {}
self.postcodes = set()
self.housenumbers = {}
Then placex contains
| object | parent_place_id |
| N1 | N2 |
- Then search_name contains
- | object | name_vector | nameaddress_vector |
- | N1 | #Walltown | Strange, Town |
When sending search query "23 Rose Street"
Then exactly 1 results are returned
And results contain
| W1 | highway | residential | Rose Street | :w-north |
| N2 | place | city | Strange Town | :p-N1 |
When importing
- Then search_name contains
- | object | name_vector | nameaddress_vector |
- | N1 | #Walltown, #Blue house | Walltown, Strange, Town |
When sending search query "23 Walltown, Strange Town"
Then results contain
| osm | display_name |
| W1 | highway | residential | Rose Street | :w-north |
| N2 | place | city | Strange Town | :p-N1 |
When importing
- Then search_name contains
- | object | name_vector | nameaddress_vector |
- | N1 | #Moon sun, #Blue house | Moon, Sun, Strange, Town |
When sending search query "23 Moon Sun, Strange Town"
Then results contain
| osm | display_name |
| W1 | highway | residential | Rose Street | Walltown | :w-north |
| N2 | place | suburb | Strange Town | Walltown | :p-N1 |
When importing
- Then search_name contains
- | object | name_vector | nameaddress_vector |
- | N1 | #Walltown | Strange, Town |
When sending search query "23 Rose Street, Walltown"
Then exactly 1 result is returned
And results contain
| W1 | highway | residential | Rose Street | :w-north |
| N2 | place | suburb | Strange Town | :p-N1 |
When importing
- Then search_name contains
- | object | name_vector | nameaddress_vector |
- | N1 | #Green Moss | Walltown |
When sending search query "Green Moss, Rose Street, Walltown"
Then exactly 0 result is returned
When sending search query "Green Moss, Walltown"
from nominatim.tokenizer.icu_name_processor import ICUNameProcessorRules
from nominatim.tokenizer.icu_rule_loader import ICURuleLoader
from nominatim.db import properties
+from nominatim.db.sql_preprocessor import SQLPreprocessor
from mock_icu_word_table import MockIcuWordTable
return _mk_analyser
+@pytest.fixture
+def sql_functions(temp_db_conn, def_config, src_dir):
+ orig_sql = def_config.lib_dir.sql
+ def_config.lib_dir.sql = src_dir / 'lib-sql'
+ sqlproc = SQLPreprocessor(temp_db_conn, def_config)
+ sqlproc.run_sql_file(temp_db_conn, 'functions/utils.sql')
+ sqlproc.run_sql_file(temp_db_conn, 'tokenizer/icu_tokenizer.sql')
+ def_config.lib_dir.sql = orig_sql
+
@pytest.fixture
def getorcreate_full_word(temp_db_cursor):
tok.init_new_db(test_config)
assert db_prop(icu_tokenizer.DBCFG_TERM_NORMALIZATION) == ':: lower();'
- assert db_prop(icu_tokenizer.DBCFG_MAXWORDFREQ) is not None
def test_init_word_table(tokenizer_factory, test_config, place_row, word_table):
def test_init_from_project(monkeypatch, test_config, tokenizer_factory):
monkeypatch.setenv('NOMINATIM_TERM_NORMALIZATION', ':: lower();')
- monkeypatch.setenv('NOMINATIM_MAX_WORD_FREQUENCY', '90300')
tok = tokenizer_factory()
tok.init_new_db(test_config)
monkeypatch.undo()
assert tok.naming_rules is not None
assert tok.term_normalization == ':: lower();'
- assert tok.max_word_frequency == '90300'
def test_update_sql_functions(db_prop, temp_db_cursor,
tokenizer_factory, test_config, table_factory,
monkeypatch):
- monkeypatch.setenv('NOMINATIM_MAX_WORD_FREQUENCY', '1133')
tok = tokenizer_factory()
tok.init_new_db(test_config)
- monkeypatch.undo()
-
- assert db_prop(icu_tokenizer.DBCFG_MAXWORDFREQ) == '1133'
table_factory('test', 'txt TEXT')
func_file = test_config.lib_dir.sql / 'tokenizer' / 'icu_tokenizer.sql'
- func_file.write_text("""INSERT INTO test VALUES ('{{max_word_freq}}')""")
+ func_file.write_text("""INSERT INTO test VALUES (1133)""")
tok.update_sql_functions(test_config)
class TestPlaceNames:
@pytest.fixture(autouse=True)
- def setup(self, analyzer, getorcreate_full_word):
+ def setup(self, analyzer, sql_functions):
with analyzer() as anl:
self.analyzer = anl
yield anl
class TestPlaceAddress:
@pytest.fixture(autouse=True)
- def setup(self, analyzer, getorcreate_full_word):
+ def setup(self, analyzer, sql_functions):
with analyzer(trans=(":: upper()", "'🜵' > ' '")) as anl:
self.analyzer = anl
yield anl
def test_process_place_street(self):
info = self.process_address(street='Grand Road')
- assert eval(info['street']) == self.name_token_set('#GRAND ROAD')
+ assert eval(info['street']) == self.name_token_set('GRAND', 'ROAD')
def test_process_place_street_empty(self):
def test_process_place_place(self):
info = self.process_address(place='Honu Lulu')
- assert eval(info['place_search']) == self.name_token_set('#HONU LULU',
- 'HONU', 'LULU')
- assert eval(info['place_match']) == self.name_token_set('#HONU LULU')
+ assert eval(info['place']) == self.name_token_set('HONU', 'LULU')
def test_process_place_place_empty(self):
info = self.process_address(place='🜵')
- assert 'place_search' not in info
- assert 'place_match' not in info
+ assert 'place' not in info
def test_process_place_address_terms(self):
suburb='Zwickau', street='Hauptstr',
full='right behind the church')
- city_full = self.name_token_set('#ZWICKAU')
- city_all = self.name_token_set('#ZWICKAU', 'ZWICKAU')
- state_full = self.name_token_set('#SACHSEN')
- state_all = self.name_token_set('#SACHSEN', 'SACHSEN')
+ city = self.name_token_set('ZWICKAU')
+ state = self.name_token_set('SACHSEN')
- result = {k: [eval(v[0]), eval(v[1])] for k,v in info['addr'].items()}
+ result = {k: eval(v) for k,v in info['addr'].items()}
- assert result == {'city': [city_all, city_full],
- 'suburb': [city_all, city_full],
- 'state': [state_all, state_full]}
+ assert result == {'city': city, 'suburb': city, 'state': state}
def test_process_place_address_terms_empty(self):