Parallelize generation of the edge-expanded-edges.
This commit is contained in:
parent
b68d79407e
commit
35550d8c0a
@ -13,6 +13,7 @@
|
|||||||
- Added conditional restriction support with `parse-conditional-restrictions=true|false` to osrm-extract. This option saves conditional turn restrictions to the .restrictions file for parsing by contract later. Added `parse-conditionals-from-now=utc time stamp` and `--time-zone-file=/path/to/file` to osrm-contract
|
- Added conditional restriction support with `parse-conditional-restrictions=true|false` to osrm-extract. This option saves conditional turn restrictions to the .restrictions file for parsing by contract later. Added `parse-conditionals-from-now=utc time stamp` and `--time-zone-file=/path/to/file` to osrm-contract
|
||||||
- Command-line tools (osrm-extract, osrm-contract, osrm-routed, etc) now return error codes and legible error messages for common problem scenarios, rather than ugly C++ crashes
|
- Command-line tools (osrm-extract, osrm-contract, osrm-routed, etc) now return error codes and legible error messages for common problem scenarios, rather than ugly C++ crashes
|
||||||
- Speed up pre-processing by only running the Lua `node_function` for nodes that have tags. Cuts OSM file parsing time in half.
|
- Speed up pre-processing by only running the Lua `node_function` for nodes that have tags. Cuts OSM file parsing time in half.
|
||||||
|
- osrm-extract now performs generation of edge-expanded-edges using all available CPUs, which should make osrm-extract significantly faster on multi-CPU machines
|
||||||
- Files
|
- Files
|
||||||
- .osrm.nodes file was renamed to .nbg_nodes and .ebg_nodes was added
|
- .osrm.nodes file was renamed to .nbg_nodes and .ebg_nodes was added
|
||||||
- Guidance
|
- Guidance
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
#include "extractor/query_node.hpp"
|
#include "extractor/query_node.hpp"
|
||||||
#include "extractor/restriction_map.hpp"
|
#include "extractor/restriction_map.hpp"
|
||||||
|
|
||||||
|
#include "util/concurrent_id_map.hpp"
|
||||||
#include "util/deallocating_vector.hpp"
|
#include "util/deallocating_vector.hpp"
|
||||||
#include "util/guidance/bearing_class.hpp"
|
#include "util/guidance/bearing_class.hpp"
|
||||||
#include "util/guidance/entry_class.hpp"
|
#include "util/guidance/entry_class.hpp"
|
||||||
@ -41,6 +42,9 @@
|
|||||||
|
|
||||||
#include <boost/filesystem/fstream.hpp>
|
#include <boost/filesystem/fstream.hpp>
|
||||||
|
|
||||||
|
#include <tbb/concurrent_unordered_map.h>
|
||||||
|
#include <tbb/concurrent_vector.h>
|
||||||
|
|
||||||
namespace osrm
|
namespace osrm
|
||||||
{
|
{
|
||||||
namespace extractor
|
namespace extractor
|
||||||
@ -167,9 +171,9 @@ class EdgeBasedGraphFactory
|
|||||||
std::size_t skipped_uturns_counter;
|
std::size_t skipped_uturns_counter;
|
||||||
std::size_t skipped_barrier_turns_counter;
|
std::size_t skipped_barrier_turns_counter;
|
||||||
|
|
||||||
std::unordered_map<util::guidance::BearingClass, BearingClassID> bearing_class_hash;
|
util::ConcurrentIDMap<util::guidance::BearingClass, BearingClassID> bearing_class_hash;
|
||||||
std::vector<BearingClassID> bearing_class_by_node_based_node;
|
std::vector<BearingClassID> bearing_class_by_node_based_node;
|
||||||
std::unordered_map<util::guidance::EntryClass, EntryClassID> entry_class_hash;
|
util::ConcurrentIDMap<util::guidance::EntryClass, EntryClassID> entry_class_hash;
|
||||||
};
|
};
|
||||||
} // namespace extractor
|
} // namespace extractor
|
||||||
} // namespace osrm
|
} // namespace osrm
|
||||||
|
@ -65,12 +65,16 @@ const constexpr char *scenario_names[] = {"Simple",
|
|||||||
|
|
||||||
class TurnLaneHandler
|
class TurnLaneHandler
|
||||||
{
|
{
|
||||||
|
using UpgradableMutex = boost::interprocess::interprocess_upgradable_mutex;
|
||||||
|
using ScopedReaderLock = boost::interprocess::sharable_lock<UpgradableMutex>;
|
||||||
|
using ScopedWriterLock = boost::interprocess::scoped_lock<UpgradableMutex>;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
typedef std::vector<TurnLaneData> LaneDataVector;
|
typedef std::vector<TurnLaneData> LaneDataVector;
|
||||||
|
|
||||||
TurnLaneHandler(const util::NodeBasedDynamicGraph &node_based_graph,
|
TurnLaneHandler(const util::NodeBasedDynamicGraph &node_based_graph,
|
||||||
std::vector<std::uint32_t> &turn_lane_offsets,
|
const std::vector<std::uint32_t> &turn_lane_offsets,
|
||||||
std::vector<TurnLaneType::Mask> &turn_lane_masks,
|
const std::vector<TurnLaneType::Mask> &turn_lane_masks,
|
||||||
LaneDescriptionMap &lane_description_map,
|
LaneDescriptionMap &lane_description_map,
|
||||||
const TurnAnalysis &turn_analysis,
|
const TurnAnalysis &turn_analysis,
|
||||||
util::guidance::LaneDataIdMap &id_map);
|
util::guidance::LaneDataIdMap &id_map);
|
||||||
@ -86,8 +90,8 @@ class TurnLaneHandler
|
|||||||
// we need to be able to look at previous intersections to, in some cases, find the correct turn
|
// we need to be able to look at previous intersections to, in some cases, find the correct turn
|
||||||
// lanes for a turn
|
// lanes for a turn
|
||||||
const util::NodeBasedDynamicGraph &node_based_graph;
|
const util::NodeBasedDynamicGraph &node_based_graph;
|
||||||
std::vector<std::uint32_t> &turn_lane_offsets;
|
const std::vector<std::uint32_t> &turn_lane_offsets;
|
||||||
std::vector<TurnLaneType::Mask> &turn_lane_masks;
|
const std::vector<TurnLaneType::Mask> &turn_lane_masks;
|
||||||
LaneDescriptionMap &lane_description_map;
|
LaneDescriptionMap &lane_description_map;
|
||||||
const TurnAnalysis &turn_analysis;
|
const TurnAnalysis &turn_analysis;
|
||||||
util::guidance::LaneDataIdMap &id_map;
|
util::guidance::LaneDataIdMap &id_map;
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
|
|
||||||
#include <boost/functional/hash.hpp>
|
#include <boost/functional/hash.hpp>
|
||||||
|
|
||||||
|
#include "util/concurrent_id_map.hpp"
|
||||||
#include "util/json_container.hpp"
|
#include "util/json_container.hpp"
|
||||||
#include "util/typedefs.hpp"
|
#include "util/typedefs.hpp"
|
||||||
|
|
||||||
@ -93,9 +94,9 @@ struct TurnLaneDescription_hash
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
typedef std::unordered_map<guidance::TurnLaneDescription,
|
typedef util::ConcurrentIDMap<guidance::TurnLaneDescription,
|
||||||
LaneDescriptionID,
|
LaneDescriptionID,
|
||||||
guidance::TurnLaneDescription_hash>
|
guidance::TurnLaneDescription_hash>
|
||||||
LaneDescriptionMap;
|
LaneDescriptionMap;
|
||||||
|
|
||||||
} // guidance
|
} // guidance
|
||||||
|
@ -31,6 +31,15 @@ void write(storage::io::FileWriter &writer,
|
|||||||
const detail::TurnDataContainerImpl<Ownership> &turn_data);
|
const detail::TurnDataContainerImpl<Ownership> &turn_data);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
struct TurnData
|
||||||
|
{
|
||||||
|
extractor::guidance::TurnInstruction turn_instruction;
|
||||||
|
LaneDataID lane_data_id;
|
||||||
|
EntryClassID entry_class_id;
|
||||||
|
util::guidance::TurnBearing pre_turn_bearing;
|
||||||
|
util::guidance::TurnBearing post_turn_bearing;
|
||||||
|
};
|
||||||
|
|
||||||
namespace detail
|
namespace detail
|
||||||
{
|
{
|
||||||
template <storage::Ownership Ownership> class TurnDataContainerImpl
|
template <storage::Ownership Ownership> class TurnDataContainerImpl
|
||||||
@ -75,17 +84,20 @@ template <storage::Ownership Ownership> class TurnDataContainerImpl
|
|||||||
|
|
||||||
// Used by EdgeBasedGraphFactory to fill data structure
|
// Used by EdgeBasedGraphFactory to fill data structure
|
||||||
template <typename = std::enable_if<Ownership == storage::Ownership::Container>>
|
template <typename = std::enable_if<Ownership == storage::Ownership::Container>>
|
||||||
void push_back(extractor::guidance::TurnInstruction turn_instruction,
|
void push_back(const TurnData &data)
|
||||||
LaneDataID lane_data_id,
|
|
||||||
EntryClassID entry_class_id,
|
|
||||||
util::guidance::TurnBearing pre_turn_bearing,
|
|
||||||
util::guidance::TurnBearing post_turn_bearing)
|
|
||||||
{
|
{
|
||||||
turn_instructions.push_back(turn_instruction);
|
turn_instructions.push_back(data.turn_instruction);
|
||||||
lane_data_ids.push_back(lane_data_id);
|
lane_data_ids.push_back(data.lane_data_id);
|
||||||
entry_class_ids.push_back(entry_class_id);
|
entry_class_ids.push_back(data.entry_class_id);
|
||||||
pre_turn_bearings.push_back(pre_turn_bearing);
|
pre_turn_bearings.push_back(data.pre_turn_bearing);
|
||||||
post_turn_bearings.push_back(post_turn_bearing);
|
post_turn_bearings.push_back(data.post_turn_bearing);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename = std::enable_if<Ownership == storage::Ownership::Container>>
|
||||||
|
void append(const std::vector<TurnData> &others)
|
||||||
|
{
|
||||||
|
std::for_each(
|
||||||
|
others.begin(), others.end(), [this](const TurnData &other) { push_back(other); });
|
||||||
}
|
}
|
||||||
|
|
||||||
friend void serialization::read<Ownership>(storage::io::FileReader &reader,
|
friend void serialization::read<Ownership>(storage::io::FileReader &reader,
|
||||||
|
57
include/util/concurrent_id_map.hpp
Normal file
57
include/util/concurrent_id_map.hpp
Normal file
@ -0,0 +1,57 @@
|
|||||||
|
#ifndef CONCURRENT_ID_MAP_HPP
|
||||||
|
#define CONCURRENT_ID_MAP_HPP
|
||||||
|
|
||||||
|
#include <boost/interprocess/sync/interprocess_upgradable_mutex.hpp>
|
||||||
|
#include <boost/interprocess/sync/scoped_lock.hpp>
|
||||||
|
#include <boost/interprocess/sync/sharable_lock.hpp>
|
||||||
|
|
||||||
|
#include <unordered_map>
|
||||||
|
|
||||||
|
namespace osrm
|
||||||
|
{
|
||||||
|
namespace util
|
||||||
|
{
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is a special purpose map for caching incrementing IDs
|
||||||
|
*/
|
||||||
|
template <typename KeyType, typename ValueType, typename HashType = std::hash<KeyType>>
|
||||||
|
struct ConcurrentIDMap
|
||||||
|
{
|
||||||
|
static_assert(std::is_unsigned<ValueType>::value, "Only unsigned integer types are supported.");
|
||||||
|
|
||||||
|
using UpgradableMutex = boost::interprocess::interprocess_upgradable_mutex;
|
||||||
|
using ScopedReaderLock = boost::interprocess::sharable_lock<UpgradableMutex>;
|
||||||
|
using ScopedWriterLock = boost::interprocess::scoped_lock<UpgradableMutex>;
|
||||||
|
|
||||||
|
std::unordered_map<KeyType, ValueType, HashType> data;
|
||||||
|
mutable UpgradableMutex mutex;
|
||||||
|
|
||||||
|
const ValueType ConcurrentFindOrAdd(const KeyType &key)
|
||||||
|
{
|
||||||
|
{
|
||||||
|
ScopedReaderLock sentry{mutex};
|
||||||
|
const auto result = data.find(key);
|
||||||
|
if (result != data.end())
|
||||||
|
{
|
||||||
|
return result->second;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
{
|
||||||
|
ScopedWriterLock sentry{mutex};
|
||||||
|
const auto result = data.find(key);
|
||||||
|
if (result != data.end())
|
||||||
|
{
|
||||||
|
return result->second;
|
||||||
|
}
|
||||||
|
const auto id = static_cast<ValueType>(data.size());
|
||||||
|
data[key] = id;
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
} // util
|
||||||
|
} // osrm
|
||||||
|
|
||||||
|
#endif // CONCURRENT_ID_MAP_HPP
|
@ -7,6 +7,7 @@
|
|||||||
#include <unordered_map>
|
#include <unordered_map>
|
||||||
#include <vector>
|
#include <vector>
|
||||||
|
|
||||||
|
#include "util/concurrent_id_map.hpp"
|
||||||
#include "util/typedefs.hpp"
|
#include "util/typedefs.hpp"
|
||||||
|
|
||||||
#include <boost/functional/hash.hpp>
|
#include <boost/functional/hash.hpp>
|
||||||
@ -97,7 +98,7 @@ class LaneTupleIdPair
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
using LaneDataIdMap = std::unordered_map<LaneTupleIdPair, LaneDataID, boost::hash<LaneTupleIdPair>>;
|
using LaneDataIdMap = ConcurrentIDMap<LaneTupleIdPair, LaneDataID, boost::hash<LaneTupleIdPair>>;
|
||||||
|
|
||||||
} // namespace guidance
|
} // namespace guidance
|
||||||
} // namespace util
|
} // namespace util
|
||||||
|
@ -30,6 +30,11 @@
|
|||||||
#include <string>
|
#include <string>
|
||||||
#include <unordered_map>
|
#include <unordered_map>
|
||||||
|
|
||||||
|
#include <tbb/blocked_range.h>
|
||||||
|
#include <tbb/parallel_for.h>
|
||||||
|
#include <tbb/pipeline.h>
|
||||||
|
#include <tbb/task_scheduler_init.h>
|
||||||
|
|
||||||
namespace osrm
|
namespace osrm
|
||||||
{
|
{
|
||||||
namespace extractor
|
namespace extractor
|
||||||
@ -319,6 +324,7 @@ void EdgeBasedGraphFactory::GenerateEdgeExpandedEdges(
|
|||||||
const std::string &turn_duration_penalties_filename,
|
const std::string &turn_duration_penalties_filename,
|
||||||
const std::string &turn_penalties_index_filename)
|
const std::string &turn_penalties_index_filename)
|
||||||
{
|
{
|
||||||
|
|
||||||
util::Log() << "Generating edge-expanded edges ";
|
util::Log() << "Generating edge-expanded edges ";
|
||||||
|
|
||||||
std::size_t node_based_edge_counter = 0;
|
std::size_t node_based_edge_counter = 0;
|
||||||
@ -362,189 +368,318 @@ void EdgeBasedGraphFactory::GenerateEdgeExpandedEdges(
|
|||||||
const auto weight_multiplier =
|
const auto weight_multiplier =
|
||||||
scripting_environment.GetProfileProperties().GetWeightMultiplier();
|
scripting_environment.GetProfileProperties().GetWeightMultiplier();
|
||||||
|
|
||||||
|
// The following block generates the edge-based-edges using a parallel processing
|
||||||
|
// pipeline. Sets of intersection IDs are batched in groups of GRAINSIZE (100)
|
||||||
|
// `generator_stage`,
|
||||||
|
// then those groups are processed in parallel `processor_stage`. Finally, results are
|
||||||
|
// appended to the various buffer vectors by the `output_stage` in the same order
|
||||||
|
// that the `generator_stage` created them in (tbb::filter::serial_in_order creates this
|
||||||
|
// guarantee). The order needs to be maintained because we depend on it later in the
|
||||||
|
// processing pipeline.
|
||||||
{
|
{
|
||||||
util::UnbufferedLog log;
|
util::UnbufferedLog log;
|
||||||
|
|
||||||
util::Percent progress(log, m_node_based_graph->GetNumberOfNodes());
|
const NodeID node_count = m_node_based_graph->GetNumberOfNodes();
|
||||||
|
util::Percent progress(log, node_count);
|
||||||
|
// This counter is used to keep track of how far along we've made it
|
||||||
|
std::uint64_t nodes_completed = 0;
|
||||||
|
|
||||||
// going over all nodes (which form the center of an intersection), we compute all
|
// going over all nodes (which form the center of an intersection), we compute all
|
||||||
// possible turns along these intersections.
|
// possible turns along these intersections.
|
||||||
for (const auto node_at_center_of_intersection :
|
|
||||||
util::irange(0u, m_node_based_graph->GetNumberOfNodes()))
|
|
||||||
{
|
|
||||||
progress.PrintStatus(node_at_center_of_intersection);
|
|
||||||
|
|
||||||
const auto shape_result =
|
NodeID current_node = 0;
|
||||||
turn_analysis.ComputeIntersectionShapes(node_at_center_of_intersection);
|
|
||||||
|
|
||||||
// all nodes in the graph are connected in both directions. We check all outgoing nodes
|
// Handle intersections in sets of 100. The pipeline below has a serial bottleneck
|
||||||
// to
|
// during the writing phase, so we want to make the parallel workers do more work
|
||||||
// find the incoming edge. This is a larger search overhead, but the cost we need to pay
|
// to give the serial final stage time to complete its tasks.
|
||||||
// to
|
const constexpr unsigned GRAINSIZE = 100;
|
||||||
// generate edges here is worth the additional search overhead.
|
|
||||||
//
|
|
||||||
// a -> b <-> c
|
|
||||||
// |
|
|
||||||
// v
|
|
||||||
// d
|
|
||||||
//
|
|
||||||
// will have:
|
|
||||||
// a: b,rev=0
|
|
||||||
// b: a,rev=1 c,rev=0 d,rev=0
|
|
||||||
// c: b,rev=0
|
|
||||||
//
|
|
||||||
// From the flags alone, we cannot determine which nodes are connected to `b` by an
|
|
||||||
// outgoing
|
|
||||||
// edge. Therefore, we have to search all connected edges for edges entering `b`
|
|
||||||
for (const EdgeID outgoing_edge :
|
|
||||||
m_node_based_graph->GetAdjacentEdgeRange(node_at_center_of_intersection))
|
|
||||||
{
|
|
||||||
const NodeID node_along_road_entering =
|
|
||||||
m_node_based_graph->GetTarget(outgoing_edge);
|
|
||||||
|
|
||||||
const auto incoming_edge = m_node_based_graph->FindEdge(
|
// First part of the pipeline generates iterator ranges of IDs in sets of GRAINSIZE
|
||||||
node_along_road_entering, node_at_center_of_intersection);
|
tbb::filter_t<void, tbb::blocked_range<NodeID>> generator_stage(
|
||||||
|
tbb::filter::serial_in_order, [&](tbb::flow_control &fc) -> tbb::blocked_range<NodeID> {
|
||||||
if (m_node_based_graph->GetEdgeData(incoming_edge).reversed)
|
if (current_node < node_count)
|
||||||
continue;
|
|
||||||
|
|
||||||
++node_based_edge_counter;
|
|
||||||
|
|
||||||
auto intersection_with_flags_and_angles =
|
|
||||||
turn_analysis.GetIntersectionGenerator().TransformIntersectionShapeIntoView(
|
|
||||||
node_along_road_entering,
|
|
||||||
incoming_edge,
|
|
||||||
shape_result.annotated_normalized_shape.normalized_shape,
|
|
||||||
shape_result.intersection_shape,
|
|
||||||
shape_result.annotated_normalized_shape.performed_merges);
|
|
||||||
|
|
||||||
auto intersection = turn_analysis.AssignTurnTypes(
|
|
||||||
node_along_road_entering, incoming_edge, intersection_with_flags_and_angles);
|
|
||||||
|
|
||||||
OSRM_ASSERT(intersection.valid(), m_coordinates[node_at_center_of_intersection]);
|
|
||||||
|
|
||||||
intersection = turn_lane_handler.assignTurnLanes(
|
|
||||||
node_along_road_entering, incoming_edge, std::move(intersection));
|
|
||||||
|
|
||||||
// the entry class depends on the turn, so we have to classify the interesction for
|
|
||||||
// every edge
|
|
||||||
const auto turn_classification = classifyIntersection(intersection);
|
|
||||||
|
|
||||||
const auto entry_class_id = [&](const util::guidance::EntryClass entry_class) {
|
|
||||||
if (0 == entry_class_hash.count(entry_class))
|
|
||||||
{
|
|
||||||
const auto id = static_cast<std::uint16_t>(entry_class_hash.size());
|
|
||||||
entry_class_hash[entry_class] = id;
|
|
||||||
return id;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
return entry_class_hash.find(entry_class)->second;
|
|
||||||
}
|
|
||||||
}(turn_classification.first);
|
|
||||||
|
|
||||||
const auto bearing_class_id =
|
|
||||||
[&](const util::guidance::BearingClass bearing_class) {
|
|
||||||
if (0 == bearing_class_hash.count(bearing_class))
|
|
||||||
{
|
|
||||||
const auto id = static_cast<std::uint32_t>(bearing_class_hash.size());
|
|
||||||
bearing_class_hash[bearing_class] = id;
|
|
||||||
return id;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
return bearing_class_hash.find(bearing_class)->second;
|
|
||||||
}
|
|
||||||
}(turn_classification.second);
|
|
||||||
bearing_class_by_node_based_node[node_at_center_of_intersection] = bearing_class_id;
|
|
||||||
|
|
||||||
for (const auto &turn : intersection)
|
|
||||||
{
|
{
|
||||||
// only keep valid turns
|
auto next_node = std::min(current_node + GRAINSIZE, node_count);
|
||||||
if (!turn.entry_allowed)
|
auto result = tbb::blocked_range<NodeID>(current_node, next_node);
|
||||||
continue;
|
current_node = next_node;
|
||||||
|
return result;
|
||||||
// only add an edge if turn is not prohibited
|
|
||||||
const EdgeData &edge_data1 = m_node_based_graph->GetEdgeData(incoming_edge);
|
|
||||||
const EdgeData &edge_data2 = m_node_based_graph->GetEdgeData(turn.eid);
|
|
||||||
|
|
||||||
BOOST_ASSERT(edge_data1.edge_id != edge_data2.edge_id);
|
|
||||||
BOOST_ASSERT(!edge_data1.reversed);
|
|
||||||
BOOST_ASSERT(!edge_data2.reversed);
|
|
||||||
|
|
||||||
// the following is the core of the loop.
|
|
||||||
turn_data_container.push_back(
|
|
||||||
turn.instruction,
|
|
||||||
turn.lane_data_id,
|
|
||||||
entry_class_id,
|
|
||||||
util::guidance::TurnBearing(intersection[0].bearing),
|
|
||||||
util::guidance::TurnBearing(turn.bearing));
|
|
||||||
|
|
||||||
// compute weight and duration penalties
|
|
||||||
auto is_traffic_light = m_traffic_lights.count(node_at_center_of_intersection);
|
|
||||||
ExtractionTurn extracted_turn(turn, is_traffic_light);
|
|
||||||
extracted_turn.source_restricted = edge_data1.restricted;
|
|
||||||
extracted_turn.target_restricted = edge_data2.restricted;
|
|
||||||
scripting_environment.ProcessTurn(extracted_turn);
|
|
||||||
|
|
||||||
// turn penalties are limited to [-2^15, 2^15) which roughly
|
|
||||||
// translates to 54 minutes and fits signed 16bit deci-seconds
|
|
||||||
auto weight_penalty =
|
|
||||||
boost::numeric_cast<TurnPenalty>(extracted_turn.weight * weight_multiplier);
|
|
||||||
auto duration_penalty =
|
|
||||||
boost::numeric_cast<TurnPenalty>(extracted_turn.duration * 10.);
|
|
||||||
|
|
||||||
BOOST_ASSERT(SPECIAL_NODEID != edge_data1.edge_id);
|
|
||||||
BOOST_ASSERT(SPECIAL_NODEID != edge_data2.edge_id);
|
|
||||||
|
|
||||||
// NOTE: potential overflow here if we hit 2^32 routable edges
|
|
||||||
BOOST_ASSERT(m_edge_based_edge_list.size() <=
|
|
||||||
std::numeric_limits<NodeID>::max());
|
|
||||||
auto turn_id = m_edge_based_edge_list.size();
|
|
||||||
auto weight =
|
|
||||||
boost::numeric_cast<EdgeWeight>(edge_data1.weight + weight_penalty);
|
|
||||||
auto duration =
|
|
||||||
boost::numeric_cast<EdgeWeight>(edge_data1.duration + duration_penalty);
|
|
||||||
m_edge_based_edge_list.emplace_back(edge_data1.edge_id,
|
|
||||||
edge_data2.edge_id,
|
|
||||||
turn_id,
|
|
||||||
weight,
|
|
||||||
duration,
|
|
||||||
true,
|
|
||||||
false);
|
|
||||||
|
|
||||||
BOOST_ASSERT(turn_weight_penalties.size() == turn_id);
|
|
||||||
turn_weight_penalties.push_back(weight_penalty);
|
|
||||||
BOOST_ASSERT(turn_duration_penalties.size() == turn_id);
|
|
||||||
turn_duration_penalties.push_back(duration_penalty);
|
|
||||||
|
|
||||||
// We write out the mapping between the edge-expanded edges and the
|
|
||||||
// original nodes. Since each edge represents a possible maneuver, external
|
|
||||||
// programs can use this to quickly perform updates to edge weights in order
|
|
||||||
// to penalize certain turns.
|
|
||||||
|
|
||||||
// If this edge is 'trivial' -- where the compressed edge corresponds
|
|
||||||
// exactly to an original OSM segment -- we can pull the turn's preceding
|
|
||||||
// node ID directly with `node_along_road_entering`; otherwise, we need to
|
|
||||||
// look up the node immediately preceding the turn from the compressed edge
|
|
||||||
// container.
|
|
||||||
const bool isTrivial = m_compressed_edge_container.IsTrivial(incoming_edge);
|
|
||||||
|
|
||||||
const auto &from_node =
|
|
||||||
isTrivial ? node_along_road_entering
|
|
||||||
: m_compressed_edge_container.GetLastEdgeSourceID(incoming_edge);
|
|
||||||
const auto &via_node =
|
|
||||||
m_compressed_edge_container.GetLastEdgeTargetID(incoming_edge);
|
|
||||||
const auto &to_node =
|
|
||||||
m_compressed_edge_container.GetFirstEdgeTargetID(turn.eid);
|
|
||||||
|
|
||||||
lookup::TurnIndexBlock turn_index_block = {from_node, via_node, to_node};
|
|
||||||
|
|
||||||
turn_penalties_index_file.WriteOne(turn_index_block);
|
|
||||||
}
|
}
|
||||||
}
|
else
|
||||||
|
{
|
||||||
|
fc.stop();
|
||||||
|
return tbb::blocked_range<NodeID>(node_count, node_count);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
// This struct is the buffered output of the `processor_stage`. This data is
|
||||||
|
// appended to the various output arrays/files by the `output_stage`.
|
||||||
|
struct IntersectionData
|
||||||
|
{
|
||||||
|
std::size_t nodes_processed = 0;
|
||||||
|
std::vector<lookup::TurnIndexBlock> turn_indexes;
|
||||||
|
std::vector<EdgeBasedEdge> edges_list;
|
||||||
|
std::vector<TurnPenalty> turn_weight_penalties;
|
||||||
|
std::vector<TurnPenalty> turn_duration_penalties;
|
||||||
|
std::vector<TurnData> turn_data_container;
|
||||||
|
};
|
||||||
|
|
||||||
|
// Second part of the pipeline is where the intersection analysis is done for
|
||||||
|
// each intersection
|
||||||
|
tbb::filter_t<tbb::blocked_range<NodeID>, std::shared_ptr<IntersectionData>>
|
||||||
|
processor_stage(
|
||||||
|
tbb::filter::parallel, [&](const tbb::blocked_range<NodeID> &intersection_node_range) {
|
||||||
|
|
||||||
|
auto buffer = std::make_shared<IntersectionData>();
|
||||||
|
buffer->nodes_processed =
|
||||||
|
intersection_node_range.end() - intersection_node_range.begin();
|
||||||
|
|
||||||
|
// If we get fed a 0-length range for some reason, we can just return right away
|
||||||
|
if (buffer->nodes_processed == 0)
|
||||||
|
return buffer;
|
||||||
|
|
||||||
|
for (auto node_at_center_of_intersection = intersection_node_range.begin(),
|
||||||
|
end = intersection_node_range.end();
|
||||||
|
node_at_center_of_intersection < end;
|
||||||
|
++node_at_center_of_intersection)
|
||||||
|
{
|
||||||
|
|
||||||
|
// We capture the thread-local work in these objects, then flush
|
||||||
|
// them in a controlled manner at the end of the parallel range
|
||||||
|
|
||||||
|
const auto shape_result =
|
||||||
|
turn_analysis.ComputeIntersectionShapes(node_at_center_of_intersection);
|
||||||
|
|
||||||
|
// all nodes in the graph are connected in both directions. We check all
|
||||||
|
// outgoing nodes to find the incoming edge. This is a larger search overhead,
|
||||||
|
// but the cost we need to pay to generate edges here is worth the additional
|
||||||
|
// search overhead.
|
||||||
|
//
|
||||||
|
// a -> b <-> c
|
||||||
|
// |
|
||||||
|
// v
|
||||||
|
// d
|
||||||
|
//
|
||||||
|
// will have:
|
||||||
|
// a: b,rev=0
|
||||||
|
// b: a,rev=1 c,rev=0 d,rev=0
|
||||||
|
// c: b,rev=0
|
||||||
|
//
|
||||||
|
// From the flags alone, we cannot determine which nodes are connected to
|
||||||
|
// `b` by an outgoing edge. Therefore, we have to search all connected edges for
|
||||||
|
// edges entering `b`
|
||||||
|
for (const EdgeID outgoing_edge :
|
||||||
|
m_node_based_graph->GetAdjacentEdgeRange(node_at_center_of_intersection))
|
||||||
|
{
|
||||||
|
const NodeID node_along_road_entering =
|
||||||
|
m_node_based_graph->GetTarget(outgoing_edge);
|
||||||
|
|
||||||
|
const auto incoming_edge = m_node_based_graph->FindEdge(
|
||||||
|
node_along_road_entering, node_at_center_of_intersection);
|
||||||
|
|
||||||
|
if (m_node_based_graph->GetEdgeData(incoming_edge).reversed)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
++node_based_edge_counter;
|
||||||
|
|
||||||
|
auto intersection_with_flags_and_angles =
|
||||||
|
turn_analysis.GetIntersectionGenerator()
|
||||||
|
.TransformIntersectionShapeIntoView(
|
||||||
|
node_along_road_entering,
|
||||||
|
incoming_edge,
|
||||||
|
shape_result.annotated_normalized_shape.normalized_shape,
|
||||||
|
shape_result.intersection_shape,
|
||||||
|
shape_result.annotated_normalized_shape.performed_merges);
|
||||||
|
|
||||||
|
auto intersection =
|
||||||
|
turn_analysis.AssignTurnTypes(node_along_road_entering,
|
||||||
|
incoming_edge,
|
||||||
|
intersection_with_flags_and_angles);
|
||||||
|
|
||||||
|
OSRM_ASSERT(intersection.valid(),
|
||||||
|
m_coordinates[node_at_center_of_intersection]);
|
||||||
|
|
||||||
|
intersection = turn_lane_handler.assignTurnLanes(
|
||||||
|
node_along_road_entering, incoming_edge, std::move(intersection));
|
||||||
|
|
||||||
|
// the entry class depends on the turn, so we have to classify the
|
||||||
|
// interesction for
|
||||||
|
// every edge
|
||||||
|
const auto turn_classification = classifyIntersection(intersection);
|
||||||
|
|
||||||
|
const auto entry_class_id =
|
||||||
|
entry_class_hash.ConcurrentFindOrAdd(turn_classification.first);
|
||||||
|
|
||||||
|
const auto bearing_class_id =
|
||||||
|
bearing_class_hash.ConcurrentFindOrAdd(turn_classification.second);
|
||||||
|
|
||||||
|
// Note - this is strictly speaking not thread safe, but we know we
|
||||||
|
// should never be touching the same element twice, so we should
|
||||||
|
// be fine.
|
||||||
|
bearing_class_by_node_based_node[node_at_center_of_intersection] =
|
||||||
|
bearing_class_id;
|
||||||
|
|
||||||
|
for (const auto &turn : intersection)
|
||||||
|
{
|
||||||
|
// only keep valid turns
|
||||||
|
if (!turn.entry_allowed)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
// only add an edge if turn is not prohibited
|
||||||
|
const EdgeData &edge_data1 =
|
||||||
|
m_node_based_graph->GetEdgeData(incoming_edge);
|
||||||
|
const EdgeData &edge_data2 = m_node_based_graph->GetEdgeData(turn.eid);
|
||||||
|
|
||||||
|
BOOST_ASSERT(edge_data1.edge_id != edge_data2.edge_id);
|
||||||
|
BOOST_ASSERT(!edge_data1.reversed);
|
||||||
|
BOOST_ASSERT(!edge_data2.reversed);
|
||||||
|
|
||||||
|
// the following is the core of the loop.
|
||||||
|
buffer->turn_data_container.push_back(
|
||||||
|
{turn.instruction,
|
||||||
|
turn.lane_data_id,
|
||||||
|
entry_class_id,
|
||||||
|
util::guidance::TurnBearing(intersection[0].bearing),
|
||||||
|
util::guidance::TurnBearing(turn.bearing)});
|
||||||
|
|
||||||
|
// compute weight and duration penalties
|
||||||
|
auto is_traffic_light =
|
||||||
|
m_traffic_lights.count(node_at_center_of_intersection);
|
||||||
|
ExtractionTurn extracted_turn(turn, is_traffic_light);
|
||||||
|
extracted_turn.source_restricted = edge_data1.restricted;
|
||||||
|
extracted_turn.target_restricted = edge_data2.restricted;
|
||||||
|
scripting_environment.ProcessTurn(extracted_turn);
|
||||||
|
|
||||||
|
// turn penalties are limited to [-2^15, 2^15) which roughly
|
||||||
|
// translates to 54 minutes and fits signed 16bit deci-seconds
|
||||||
|
auto weight_penalty = boost::numeric_cast<TurnPenalty>(
|
||||||
|
extracted_turn.weight * weight_multiplier);
|
||||||
|
auto duration_penalty =
|
||||||
|
boost::numeric_cast<TurnPenalty>(extracted_turn.duration * 10.);
|
||||||
|
|
||||||
|
BOOST_ASSERT(SPECIAL_NODEID != edge_data1.edge_id);
|
||||||
|
BOOST_ASSERT(SPECIAL_NODEID != edge_data2.edge_id);
|
||||||
|
|
||||||
|
// auto turn_id = m_edge_based_edge_list.size();
|
||||||
|
auto weight =
|
||||||
|
boost::numeric_cast<EdgeWeight>(edge_data1.weight + weight_penalty);
|
||||||
|
auto duration = boost::numeric_cast<EdgeWeight>(edge_data1.duration +
|
||||||
|
duration_penalty);
|
||||||
|
buffer->edges_list.emplace_back(
|
||||||
|
edge_data1.edge_id,
|
||||||
|
edge_data2.edge_id,
|
||||||
|
SPECIAL_NODEID, // This will be updated once the main loop
|
||||||
|
// completes!
|
||||||
|
weight,
|
||||||
|
duration,
|
||||||
|
true,
|
||||||
|
false);
|
||||||
|
|
||||||
|
BOOST_ASSERT(buffer->turn_weight_penalties.size() ==
|
||||||
|
buffer->edges_list.size() - 1);
|
||||||
|
buffer->turn_weight_penalties.push_back(weight_penalty);
|
||||||
|
BOOST_ASSERT(buffer->turn_duration_penalties.size() ==
|
||||||
|
buffer->edges_list.size() - 1);
|
||||||
|
buffer->turn_duration_penalties.push_back(duration_penalty);
|
||||||
|
|
||||||
|
// We write out the mapping between the edge-expanded edges and the
|
||||||
|
// original nodes. Since each edge represents a possible maneuver,
|
||||||
|
// external programs can use this to quickly perform updates to edge
|
||||||
|
// weights in order to penalize certain turns.
|
||||||
|
|
||||||
|
// If this edge is 'trivial' -- where the compressed edge corresponds
|
||||||
|
// exactly to an original OSM segment -- we can pull the turn's
|
||||||
|
// preceding node ID directly with `node_along_road_entering`;
|
||||||
|
// otherwise, we need to look up the node immediately preceding the turn
|
||||||
|
// from the compressed edge container.
|
||||||
|
const bool isTrivial =
|
||||||
|
m_compressed_edge_container.IsTrivial(incoming_edge);
|
||||||
|
|
||||||
|
const auto &from_node =
|
||||||
|
isTrivial ? node_along_road_entering
|
||||||
|
: m_compressed_edge_container.GetLastEdgeSourceID(
|
||||||
|
incoming_edge);
|
||||||
|
const auto &via_node =
|
||||||
|
m_compressed_edge_container.GetLastEdgeTargetID(incoming_edge);
|
||||||
|
const auto &to_node =
|
||||||
|
m_compressed_edge_container.GetFirstEdgeTargetID(turn.eid);
|
||||||
|
|
||||||
|
buffer->turn_indexes.push_back({from_node, via_node, to_node});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return buffer;
|
||||||
|
});
|
||||||
|
|
||||||
|
// Because we write TurnIndexBlock data as we go, we'll
|
||||||
|
// buffer them into groups of 1000 to reduce the syscall
|
||||||
|
// count by 1000x. This doesn't need much memory, but
|
||||||
|
// greatly reduces the syscall overhead of writing lots
|
||||||
|
// of small objects
|
||||||
|
const constexpr int TURN_INDEX_WRITE_BUFFER_SIZE = 1000;
|
||||||
|
std::vector<lookup::TurnIndexBlock> turn_indexes_write_buffer;
|
||||||
|
turn_indexes_write_buffer.reserve(TURN_INDEX_WRITE_BUFFER_SIZE);
|
||||||
|
|
||||||
|
// Last part of the pipeline puts all the calculated data into the serial buffers
|
||||||
|
tbb::filter_t<std::shared_ptr<IntersectionData>, void> output_stage(
|
||||||
|
tbb::filter::serial_in_order, [&](const std::shared_ptr<IntersectionData> buffer) {
|
||||||
|
|
||||||
|
nodes_completed += buffer->nodes_processed;
|
||||||
|
progress.PrintStatus(nodes_completed);
|
||||||
|
|
||||||
|
// NOTE: potential overflow here if we hit 2^32 routable edges
|
||||||
|
m_edge_based_edge_list.append(buffer->edges_list.begin(), buffer->edges_list.end());
|
||||||
|
BOOST_ASSERT(m_edge_based_edge_list.size() <= std::numeric_limits<NodeID>::max());
|
||||||
|
|
||||||
|
turn_weight_penalties.insert(turn_weight_penalties.end(),
|
||||||
|
buffer->turn_weight_penalties.begin(),
|
||||||
|
buffer->turn_weight_penalties.end());
|
||||||
|
turn_duration_penalties.insert(turn_duration_penalties.end(),
|
||||||
|
buffer->turn_duration_penalties.begin(),
|
||||||
|
buffer->turn_duration_penalties.end());
|
||||||
|
turn_data_container.append(buffer->turn_data_container);
|
||||||
|
|
||||||
|
turn_indexes_write_buffer.insert(turn_indexes_write_buffer.end(),
|
||||||
|
buffer->turn_indexes.begin(),
|
||||||
|
buffer->turn_indexes.end());
|
||||||
|
|
||||||
|
// Buffer writes to reduce syscall count
|
||||||
|
if (turn_indexes_write_buffer.size() >= TURN_INDEX_WRITE_BUFFER_SIZE)
|
||||||
|
{
|
||||||
|
turn_penalties_index_file.WriteFrom(turn_indexes_write_buffer.data(),
|
||||||
|
turn_indexes_write_buffer.size());
|
||||||
|
turn_indexes_write_buffer.clear();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
// Now, execute the pipeline. The value of "5" here was chosen by experimentation
|
||||||
|
// on a 16-CPU machine and seemed to give the best performance. This value needs
|
||||||
|
// to be balanced with the GRAINSIZE above - ideally, the pipeline puts as much work
|
||||||
|
// as possible in the `intersection_handler` step so that those parallel workers don't
|
||||||
|
// get blocked too much by the slower (io-performing) `buffer_storage`
|
||||||
|
tbb::parallel_pipeline(tbb::task_scheduler_init::default_num_threads() * 5,
|
||||||
|
generator_stage & processor_stage & output_stage);
|
||||||
|
|
||||||
|
// Flush the turn_indexes_write_buffer if it's not empty
|
||||||
|
if (!turn_indexes_write_buffer.empty())
|
||||||
|
{
|
||||||
|
turn_penalties_index_file.WriteFrom(turn_indexes_write_buffer.data(),
|
||||||
|
turn_indexes_write_buffer.size());
|
||||||
|
turn_indexes_write_buffer.clear();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
util::Log() << "Reunmbering turns";
|
||||||
|
// Now, update the turn_id property on every EdgeBasedEdge - it will equal the
|
||||||
|
// position in the m_edge_based_edge_list array for each object.
|
||||||
|
tbb::parallel_for(tbb::blocked_range<NodeID>(0, m_edge_based_edge_list.size()),
|
||||||
|
[this](const tbb::blocked_range<NodeID> &range) {
|
||||||
|
for (auto x = range.begin(), end = range.end(); x != end; ++x)
|
||||||
|
{
|
||||||
|
m_edge_based_edge_list[x].data.turn_id = x;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
// write weight penalties per turn
|
// write weight penalties per turn
|
||||||
BOOST_ASSERT(turn_weight_penalties.size() == turn_duration_penalties.size());
|
BOOST_ASSERT(turn_weight_penalties.size() == turn_duration_penalties.size());
|
||||||
{
|
{
|
||||||
@ -559,17 +694,17 @@ void EdgeBasedGraphFactory::GenerateEdgeExpandedEdges(
|
|||||||
storage::serialization::write(writer, turn_duration_penalties);
|
storage::serialization::write(writer, turn_duration_penalties);
|
||||||
}
|
}
|
||||||
|
|
||||||
util::Log() << "Created " << entry_class_hash.size() << " entry classes and "
|
util::Log() << "Created " << entry_class_hash.data.size() << " entry classes and "
|
||||||
<< bearing_class_hash.size() << " Bearing Classes";
|
<< bearing_class_hash.data.size() << " Bearing Classes";
|
||||||
|
|
||||||
util::Log() << "Writing Turn Lane Data to File...";
|
util::Log() << "Writing Turn Lane Data to File...";
|
||||||
{
|
{
|
||||||
storage::io::FileWriter writer(turn_lane_data_filename,
|
storage::io::FileWriter writer(turn_lane_data_filename,
|
||||||
storage::io::FileWriter::GenerateFingerprint);
|
storage::io::FileWriter::GenerateFingerprint);
|
||||||
|
|
||||||
std::vector<util::guidance::LaneTupleIdPair> lane_data(lane_data_map.size());
|
std::vector<util::guidance::LaneTupleIdPair> lane_data(lane_data_map.data.size());
|
||||||
// extract lane data sorted by ID
|
// extract lane data sorted by ID
|
||||||
for (auto itr : lane_data_map)
|
for (auto itr : lane_data_map.data)
|
||||||
lane_data[itr.second] = itr.first;
|
lane_data[itr.second] = itr.first;
|
||||||
|
|
||||||
storage::serialization::write(writer, lane_data);
|
storage::serialization::write(writer, lane_data);
|
||||||
@ -591,8 +726,8 @@ void EdgeBasedGraphFactory::GenerateEdgeExpandedEdges(
|
|||||||
|
|
||||||
std::vector<util::guidance::BearingClass> EdgeBasedGraphFactory::GetBearingClasses() const
|
std::vector<util::guidance::BearingClass> EdgeBasedGraphFactory::GetBearingClasses() const
|
||||||
{
|
{
|
||||||
std::vector<util::guidance::BearingClass> result(bearing_class_hash.size());
|
std::vector<util::guidance::BearingClass> result(bearing_class_hash.data.size());
|
||||||
for (const auto &pair : bearing_class_hash)
|
for (const auto &pair : bearing_class_hash.data)
|
||||||
{
|
{
|
||||||
BOOST_ASSERT(pair.second < result.size());
|
BOOST_ASSERT(pair.second < result.size());
|
||||||
result[pair.second] = pair.first;
|
result[pair.second] = pair.first;
|
||||||
@ -612,8 +747,8 @@ std::vector<BearingClassID> &EdgeBasedGraphFactory::GetBearingClassIds()
|
|||||||
|
|
||||||
std::vector<util::guidance::EntryClass> EdgeBasedGraphFactory::GetEntryClasses() const
|
std::vector<util::guidance::EntryClass> EdgeBasedGraphFactory::GetEntryClasses() const
|
||||||
{
|
{
|
||||||
std::vector<util::guidance::EntryClass> result(entry_class_hash.size());
|
std::vector<util::guidance::EntryClass> result(entry_class_hash.data.size());
|
||||||
for (const auto &pair : entry_class_hash)
|
for (const auto &pair : entry_class_hash.data)
|
||||||
{
|
{
|
||||||
BOOST_ASSERT(pair.second < result.size());
|
BOOST_ASSERT(pair.second < result.size());
|
||||||
result[pair.second] = pair.first;
|
result[pair.second] = pair.first;
|
||||||
|
@ -77,8 +77,9 @@ transformTurnLaneMapIntoArrays(const guidance::LaneDescriptionMap &turn_lane_map
|
|||||||
//
|
//
|
||||||
// turn lane offsets points into the locations of the turn_lane_masks array. We use a standard
|
// turn lane offsets points into the locations of the turn_lane_masks array. We use a standard
|
||||||
// adjacency array like structure to store the turn lane masks.
|
// adjacency array like structure to store the turn lane masks.
|
||||||
std::vector<std::uint32_t> turn_lane_offsets(turn_lane_map.size() + 2); // empty ID + sentinel
|
std::vector<std::uint32_t> turn_lane_offsets(turn_lane_map.data.size() +
|
||||||
for (auto entry = turn_lane_map.begin(); entry != turn_lane_map.end(); ++entry)
|
2); // empty ID + sentinel
|
||||||
|
for (auto entry = turn_lane_map.data.begin(); entry != turn_lane_map.data.end(); ++entry)
|
||||||
turn_lane_offsets[entry->second + 1] = entry->first.size();
|
turn_lane_offsets[entry->second + 1] = entry->first.size();
|
||||||
|
|
||||||
// inplace prefix sum
|
// inplace prefix sum
|
||||||
@ -86,7 +87,7 @@ transformTurnLaneMapIntoArrays(const guidance::LaneDescriptionMap &turn_lane_map
|
|||||||
|
|
||||||
// allocate the current masks
|
// allocate the current masks
|
||||||
std::vector<guidance::TurnLaneType::Mask> turn_lane_masks(turn_lane_offsets.back());
|
std::vector<guidance::TurnLaneType::Mask> turn_lane_masks(turn_lane_offsets.back());
|
||||||
for (auto entry = turn_lane_map.begin(); entry != turn_lane_map.end(); ++entry)
|
for (auto entry = turn_lane_map.data.begin(); entry != turn_lane_map.data.end(); ++entry)
|
||||||
std::copy(entry->first.begin(),
|
std::copy(entry->first.begin(),
|
||||||
entry->first.end(),
|
entry->first.end(),
|
||||||
turn_lane_masks.begin() + turn_lane_offsets[entry->second]);
|
turn_lane_masks.begin() + turn_lane_offsets[entry->second]);
|
||||||
@ -331,7 +332,7 @@ std::vector<TurnRestriction> Extractor::ParseOSMData(ScriptingEnvironment &scrip
|
|||||||
<< " ways, and " << number_of_relations << " relations";
|
<< " ways, and " << number_of_relations << " relations";
|
||||||
|
|
||||||
// take control over the turn lane map
|
// take control over the turn lane map
|
||||||
turn_lane_map = extractor_callbacks->moveOutLaneDescriptionMap();
|
turn_lane_map.data = extractor_callbacks->moveOutLaneDescriptionMap().data;
|
||||||
|
|
||||||
extractor_callbacks.reset();
|
extractor_callbacks.reset();
|
||||||
|
|
||||||
|
@ -41,7 +41,7 @@ ExtractorCallbacks::ExtractorCallbacks(ExtractionContainers &extraction_containe
|
|||||||
{
|
{
|
||||||
// we reserved 0, 1, 2, 3 for the empty case
|
// we reserved 0, 1, 2, 3 for the empty case
|
||||||
string_map[MapKey("", "", "", "")] = 0;
|
string_map[MapKey("", "", "", "")] = 0;
|
||||||
lane_description_map[TurnLaneDescription()] = 0;
|
lane_description_map.data[TurnLaneDescription()] = 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -251,18 +251,7 @@ void ExtractorCallbacks::ProcessWay(const osmium::Way &input_way, const Extracti
|
|||||||
return INVALID_LANE_DESCRIPTIONID;
|
return INVALID_LANE_DESCRIPTIONID;
|
||||||
TurnLaneDescription lane_description = laneStringToDescription(std::move(lane_string));
|
TurnLaneDescription lane_description = laneStringToDescription(std::move(lane_string));
|
||||||
|
|
||||||
const auto lane_description_itr = lane_description_map.find(lane_description);
|
return lane_description_map.ConcurrentFindOrAdd(lane_description);
|
||||||
if (lane_description_itr == lane_description_map.end())
|
|
||||||
{
|
|
||||||
const LaneDescriptionID new_id =
|
|
||||||
boost::numeric_cast<LaneDescriptionID>(lane_description_map.size());
|
|
||||||
lane_description_map[lane_description] = new_id;
|
|
||||||
return new_id;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
return lane_description_itr->second;
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
// Deduplicates street names, refs, destinations, pronunciation based on the string_map.
|
// Deduplicates street names, refs, destinations, pronunciation based on the string_map.
|
||||||
|
@ -34,8 +34,8 @@ std::size_t getNumberOfTurns(const Intersection &intersection)
|
|||||||
} // namespace
|
} // namespace
|
||||||
|
|
||||||
TurnLaneHandler::TurnLaneHandler(const util::NodeBasedDynamicGraph &node_based_graph,
|
TurnLaneHandler::TurnLaneHandler(const util::NodeBasedDynamicGraph &node_based_graph,
|
||||||
std::vector<std::uint32_t> &turn_lane_offsets,
|
const std::vector<std::uint32_t> &turn_lane_offsets,
|
||||||
std::vector<TurnLaneType::Mask> &turn_lane_masks,
|
const std::vector<TurnLaneType::Mask> &turn_lane_masks,
|
||||||
LaneDescriptionMap &lane_description_map,
|
LaneDescriptionMap &lane_description_map,
|
||||||
const TurnAnalysis &turn_analysis,
|
const TurnAnalysis &turn_analysis,
|
||||||
util::guidance::LaneDataIdMap &id_map)
|
util::guidance::LaneDataIdMap &id_map)
|
||||||
@ -781,19 +781,8 @@ Intersection TurnLaneHandler::handleSliproadTurn(Intersection intersection,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto combined_id = [&]() {
|
const auto combined_id = lane_description_map.ConcurrentFindOrAdd(combined_description);
|
||||||
auto itr = lane_description_map.find(combined_description);
|
|
||||||
if (lane_description_map.find(combined_description) == lane_description_map.end())
|
|
||||||
{
|
|
||||||
const auto new_id = boost::numeric_cast<LaneDescriptionID>(lane_description_map.size());
|
|
||||||
lane_description_map[combined_description] = new_id;
|
|
||||||
return new_id;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
return itr->second;
|
|
||||||
}
|
|
||||||
}();
|
|
||||||
return simpleMatchTuplesToTurns(std::move(intersection), lane_data, combined_id);
|
return simpleMatchTuplesToTurns(std::move(intersection), lane_data, combined_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -209,16 +209,7 @@ Intersection triviallyMatchLanesToTurns(Intersection intersection,
|
|||||||
util::guidance::LaneTupleIdPair key{{LaneID(data.to - data.from + 1), data.from},
|
util::guidance::LaneTupleIdPair key{{LaneID(data.to - data.from + 1), data.from},
|
||||||
lane_string_id};
|
lane_string_id};
|
||||||
|
|
||||||
auto lane_data_id = boost::numeric_cast<LaneDataID>(lane_data_to_id.size());
|
road.lane_data_id = lane_data_to_id.ConcurrentFindOrAdd(key);
|
||||||
const auto it = lane_data_to_id.find(key);
|
|
||||||
|
|
||||||
if (it == lane_data_to_id.end())
|
|
||||||
lane_data_to_id.insert({key, lane_data_id});
|
|
||||||
else
|
|
||||||
lane_data_id = it->second;
|
|
||||||
|
|
||||||
// set lane id instead after the switch:
|
|
||||||
road.lane_data_id = lane_data_id;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
if (!lane_data.empty() && lane_data.front().tag == TurnLaneType::uturn)
|
if (!lane_data.empty() && lane_data.front().tag == TurnLaneType::uturn)
|
||||||
|
Loading…
Reference in New Issue
Block a user