Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[IOTDB-144]meta data cache for query #262

Merged
merged 8 commits into from Jul 25, 2019
Merged

[IOTDB-144]meta data cache for query #262

merged 8 commits into from Jul 25, 2019

Conversation

little-emotion
Copy link
Member

To increase query speed, metadata needs to be cached, including TsFileMetaData and TsDeviceMetaData.
The files in IOTDB are organized according to time, so the query frequency of different files varies greatly. Cache metadatas of frequently queried files can reduce the time of reading metaData from disk. Besides, all sensors' metadata of a device is in TsDeviceMetaData. In order to read the metadata of a sensor, we need to read the entire TsDeviceMetaData. In the current version, querying multiple sensors in a device simultaneously need to read TsDeviceMetaData multiple times, so consider caching TsDeviceMetaData.

Copy link
Contributor

@LeiRui LeiRui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For code polish suggestions:

  1. the creation of TsFileSequenceReader in the four classes of the resourceRelated package can be moved backward to just above the creation of ChunkLoaderImpl since DeviceMetaDataCache.getInstance().get instead of MetadataQuerierByFileImpl is used to prepare metaDataList.
  2. The method TsFileMetadataUtils.getTsRowGroupBlockMetaData. Since the concept of rowgroup is out of date, why not name it getTsDeviceMetaData?

if (chunkMetaDataSize == 0 && !value.isEmpty()) {
chunkMetaDataSize = RamUsageEstimator.sizeOf(value.get(0));
}
return value.size() * chunkMetaDataSize + key.length() * 2;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why +key.length() * 2 here?
It is ChunkMetaData not TsFileMetaData.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I mistake. It is right.

/**
* This linked hash set records the access order of sensors used by query.
*/
private LinkedHashSet<String> lruForSensorUsedInQuery = new LinkedHashSet<>();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does LinkedHashSet has LRU function? Maybe you want to use LruLinkedHashMap but that is not set.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

@@ -458,6 +470,24 @@ public QueryDataSource query(String deviceId, String measurementId, QueryContext
}
}

/**
* returns the top k% measurements which are most frequently used in queries.
Copy link
Contributor

@LeiRui LeiRui Jul 21, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think the current method behaves as this annotation reads.
The current lruForSensorUsedInQuery is LinkedHashSet, which only records the access order of sensors used by query.
Even some kind of LruLinkedHashSet is used, it can only return the first num oldeset sensors kept in the LRU, not the most frequently used ones.

For example, I query d0.s0, d0.s1, d1.s2, d0.s3, d1.s1, d2.s1, d3.s1, d4.s2. The sensor are s0,s1,s2,s3,s1,s1,s1,s2. Suppose num=3

  1. LinkedHashSet
    returns s0,s1,s2
  2. some kind of LruLinkedHashSet. Suppose memory is big enough.
    returns s0,s3,s1. The top 3 of most recently used measurements.

But the top 3 of most frequently used measurements should be s1,s2,s3/s0.

Maybe you should think twice about the design, most frequently used measurements or most recently used measurements? Or as I comment in the next review, most recently used device?

most frequenly used / most recently used + measurements / devices / device.measurement
So we have 6 possible plans:

  1. most frequenly used measurements
  2. most recently used measurements
  3. most frequenly used devices
  4. most recently used devices
  5. most frequenly used device.measurement
  6. most recently used device.measurement

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. It's a bug in code. I wanted to record most recently used measurements. So I have changed LinkedHashSet to LinkedList.
  2. The load of the database will change over time, and the measurements of this query must be the recently inserted to lruForSensorUsedInQuery , so I prefer to use the recently used measurements.
  3. When querying the metadata of a device, IOTDB will only query the metadata of the device query needs, and the metadata of other devices will not be queried. So we can only cache metadata of the devices used for querying, instead of caching hot devices.
  4. Cartesian product of device and measurement is memory unacceptable

@@ -447,6 +456,9 @@ public void putAllWorkingTsFileProcessorIntoClosingList() {
// TODO need a read lock, please consider the concurrency with flush manager threads.
public QueryDataSource query(String deviceId, String measurementId, QueryContext context) {
insertLock.readLock().lock();
synchronized (lruForSensorUsedInQuery) {
lruForSensorUsedInQuery.add(measurementId);
Copy link
Contributor

@LeiRui LeiRui Jul 21, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as above, the current lruForSensorUsedInQuery is for measurementId only. My questions are:

  1. Why not use deviceId.measurementId?
    For example, root.vehicle.d0.s0 and root.vehicle.d1.s0 are different series. However, by the current code, only s0 is used. The hottest sensor doesn't equal the hottest series.

  2. Take it further, why not use deviceId
    and cache TsDeviceMetadata for every recently used device? (Or frequenly used device, but I prefer recently used device here. )
    For example, I query d0.s0, d0.s1, d0.s2, d0.s3 and d0 is the most recently used device.
    One benefit is that if the TsDeviceMetadata of file1.d0 is cached and file1.d0.s4 is not found in this TsDeviceMetadata, then file1 can be skipped safely with knowing that d0.s4 doesn't exist in this file. However, in the current pr, suppose that List<ChunkMetadata> of file1.d0.s1file1.d0.s2file1.d0.s3 is cached separately and file1.d0.s4 is not found in the cache, then file1 needs to be read again for d0.s4.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. Cartesian product of device and measurement is memory unacceptable. If Cartesian product is not cached, the corresponding relationship between device and measurement will certainly be lost, which is unavoidable.
  2. One reason for choosing to cache measurement metadata was that its replacement granularity was smaller than caching TsDeviceMetaData. Correspondingly, it is more friendly to the application scenario of querying only a small part of measurement for each device. In addition, TsFileMetaData can also filter out the tsfile without a measurement. But when some devices in tsfile have the measurements, and others don't, the problem you're talking about may arise.Both ways have their own shortcomings.

Copy link
Contributor

@LeiRui LeiRui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Two small modification suggestions.

@@ -147,6 +150,12 @@
*/
private ModificationFile mergingModification;

/**
* This linked hash set records the access order of sensors used by query.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

linked list

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

*/
private LinkedHashMap<String, TsDeviceMetadata> lruCache;
private LruLinkedHashMap<String, List<ChunkMetaData>> lruCache;

private AtomicLong cacheHintNum = new AtomicLong();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hit or hint?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hit.It's a typo.

Copy link
Contributor

@LeiRui LeiRui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is another cacheHitNum.

*/
private ConcurrentHashMap<String, TsFileMetaData> cache;
private LruLinkedHashMap<String, TsFileMetaData> cache;
private AtomicLong cacheHintNum = new AtomicLong();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hit

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

TsDeviceMetadata blockMetaData = TsFileMetadataUtils
.getTsRowGroupBlockMetaData(filePath, deviceId,
.getTsDeviceMetaData(filePath, seriesPath.getDevice(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

seriesPath.getMeasurement can be used in the getTsDeviceMetaData to help filtering by adding the following logic:

if (!fileMetaData.getMeasurementSchema().containsKey(measurementId)) {
      return null;
    }

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

Copy link
Contributor

@LeiRui LeiRui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let me sum up the latest four reviews for your convenience:

  • annotation: linked list, not set (StorageGroupProcessor)
  • name: cacheHitNum not cacheHintNum (TsFileMetaDataCach and DeviceMetaDataCache are covered.)
  • logic: If metaDataList size equals 0, then the creation of TsFileSequenceReader can be skipped. (UnseqResourceMergeReader and UnseqResourceReaderByTimestamp are covered.)
  • logic: include the filter of measurement in the TsFileMetadataUtils.getTsDeviceMetaData

metaDataList = tsFileResource.getChunkMetaDatas();
}

// create and add ChunkReader with priority
TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
.get(tsFileResource.getFile().getPath(), tsFileResource.isClosed());
ChunkLoaderImpl chunkLoader = new ChunkLoaderImpl(tsFileReader);
for (ChunkMetaData chunkMetaData : metaDataList) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If metaDataList size equals 0, then the creation of TsFileSequenceReader can be skipped.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

/**
* This class is an LRU cache. <b>Note: It's not thread safe.</b>
*/
public abstract class LruLinkedHashMap<K, V> extends LinkedHashMap<K, V> {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
public abstract class LruLinkedHashMap<K, V> extends LinkedHashMap<K, V> {
public abstract class LRULinkedHashMap<K, V> extends LinkedHashMap<K, V> {

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@qiaojialin actually I don't like the function suggested change. Look here, you changed the class name but forget to change the file name, which Suyue doesn't notice too.
I personally prefer you just point out where the problem is instead of changing it for her.


/**
* This class is copied from apache lucene, version 4.6.1. Estimates the size(memory representation)
* of Java objects. https://github.com/apache/lucene-solr/blob/releases/lucene-solr/4.6.1/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be listed in the License file

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

@qiaojialin qiaojialin merged commit 126eac7 into master Jul 25, 2019
@qiaojialin qiaojialin deleted the matadata_cache branch July 25, 2019 01:04
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants