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