This commit is contained in:
Evgeniy Gatov 2015-02-25 23:56:53 +03:00
commit 2935f975ea
50 changed files with 5402 additions and 261 deletions

View File

@ -1,17 +1,15 @@
#!/bin/bash
ck="clickhouse-client "
test_table="hits_100m"
claster="self"
start_date="'2013-07-01'"
early_stop_date="'2013-07-02'"
stop_date="'2013-07-31'"
counter_id=34
function run_ck_server
function run_ck_server
{
sudo sh -c " ulimit -v 54000000; /etc/init.d/clickhouse-server-metrika-yandex-ulimit restart"
sudo sh -c " ulimit -v 54000000; /etc/init.d/clickhouse-server restart"
}
# execute queries
@ -32,7 +30,7 @@ function execute()
if [[ $query =~ $comment_re ]]; then
echo "$query"
echo
else
else
sync
sudo sh -c "echo 3 > /proc/sys/vm/drop_caches"
@ -46,7 +44,7 @@ function execute()
fi
# restart clickhouse if failed
ps aux | grep -P '\d+ /usr/bin/clickhouse-server'
ps aux | grep -P '\d+ clickhouse-server'
if [ "$?" != "0" ]; then
run_ck_server
fi
@ -54,6 +52,7 @@ function execute()
fi
let "index = $index + 1"
echo "Ran $index queries." >&2
done
}

View File

@ -1,4 +1,3 @@
#!/bin/bash
#!/bin/expect
# Set timeout

View File

@ -4,7 +4,6 @@
#include <type_traits>
#include <stats/UniquesHashSet.h>
#include <statdaemons/HyperLogLogCounter.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
@ -15,6 +14,7 @@
#include <DB/Interpreters/AggregationCommon.h>
#include <DB/Common/HashTable/HashSet.h>
#include <DB/Common/HyperLogLogWithSmallSetOptimization.h>
#include <DB/Columns/ColumnString.h>
@ -55,16 +55,26 @@ struct AggregateFunctionUniqUniquesHashSetData
{
typedef UniquesHashSet<DefaultHash<UInt64>> Set;
Set set;
static String getName() { return "uniq"; }
};
template <typename T>
struct AggregateFunctionUniqHLL12Data
{
typedef HLL12 Set;
typedef HyperLogLogWithSmallSetOptimization<T, 16, 12> Set;
Set set;
static String getName() { return "uniqHLL12"; }
};
template <>
struct AggregateFunctionUniqHLL12Data<String>
{
typedef HyperLogLogWithSmallSetOptimization<UInt64, 16, 12> Set;
Set set;
static String getName() { return "uniqHLL12"; }
};

View File

@ -3,6 +3,8 @@
#include <DB/Common/Throttler.h>
#include <DB/Client/Connection.h>
#include <DB/Client/ConnectionPool.h>
#include <Poco/ScopedLock.h>
#include <Poco/Mutex.h>
namespace DB
@ -11,6 +13,7 @@ namespace DB
/** Для получения данных сразу из нескольких реплик (соединений) в рамках одного потока.
* В качестве вырожденного случая, может также работать с одним соединением.
* Предполагается, что все функции кроме sendCancel всегда выполняются в одном потоке.
*
* Интерфейс почти совпадает с Connection.
*/
@ -49,31 +52,36 @@ public:
std::string dumpAddresses() const;
/// Возвращает количесто реплик.
/// Без блокировки, потому что sendCancel() не меняет это количество.
size_t size() const { return replica_map.size(); }
/// Проверить, есть ли действительные реплики.
/// Без блокировки, потому что sendCancel() не меняет состояние реплик.
bool hasActiveReplicas() const { return active_replica_count > 0; }
private:
/// Реплики хэшированные по id сокета
using ReplicaMap = std::unordered_map<int, Connection *>;
private:
/// Зарегистрировать реплику.
void registerReplica(Connection * connection);
/// Внутренняя версия функции receivePacket без блокировки.
Connection::Packet receivePacketUnlocked();
/// Получить реплику, на которой можно прочитать данные.
ReplicaMap::iterator getReplicaForReading();
/** Проверить, есть ли данные, которые можно прочитать на каких-нибудь репликах.
* Возвращает одну такую реплику, если она найдётся.
*/
* Возвращает одну такую реплику, если она найдётся.
*/
ReplicaMap::iterator waitForReadEvent();
/// Пометить реплику как недействительную.
void invalidateReplica(ReplicaMap::iterator it);
private:
Settings * settings;
ReplicaMap replica_map;
@ -91,6 +99,10 @@ private:
bool sent_query = false;
/// Отменили запрос
bool cancelled = false;
/// Мьютекс для того, чтобы функция sendCancel могла выполняться безопасно
/// в отдельном потоке.
mutable Poco::FastMutex cancel_mutex;
};
}

View File

@ -63,7 +63,7 @@ private:
size_t growth_factor;
size_t linear_growth_threshold;
/// Последний непрерывный кусок памяти.
Chunk * head;
size_t size_in_bytes;
@ -77,7 +77,7 @@ private:
size_t nextSize(size_t min_next_size) const
{
size_t size_after_grow = 0;
if (head->size() < linear_growth_threshold)
size_after_grow = head->size() * growth_factor;
else
@ -119,6 +119,41 @@ public:
return res;
}
/** Отменить только что сделанное выделение памяти.
* Нужно передать размер не меньше того, который был только что выделен.
*/
void rollback(size_t size)
{
head->pos -= size;
}
/** Начать или расширить непрерывный кусок памяти.
* begin - текущее начало куска памяти, если его надо расширить, или nullptr, если его надо начать.
* Если в чанке не хватило места - скопировать существующие данные в новый кусок памяти и изменить значение begin.
*/
char * allocContinue(size_t size, char const *& begin)
{
if (unlikely(head->pos + size > head->end))
{
char * prev_end = head->pos;
addChunk(size);
if (begin)
{
begin = insert(begin, prev_end - begin);
return allocContinue(size, begin);
}
}
char * res = head->pos;
head->pos += size;
if (!begin)
begin = res;
return res;
}
/// Вставить строку без выравнивания.
const char * insert(const char * data, size_t size)
{
@ -126,7 +161,7 @@ public:
memcpy(res, data, size);
return res;
}
/// Размер выделенного пула в байтах
size_t size() const
{

View File

@ -0,0 +1,135 @@
#pragma once
#include <statdaemons/HyperLogLogCounter.h>
#include <DB/Common/HashTable/SmallTable.h>
namespace DB
{
/** Для маленького количества ключей - массив фиксированного размера "на стеке".
* Для большого - выделяется HyperLogLog.
* NOTE Возможно, имеет смысл сделать реализацию для среднего размера в виде хэш-таблицы.
*/
template <
typename Key,
UInt8 small_set_size,
UInt8 K,
typename Hash = IntHash32<Key>,
typename DenominatorType = float>
class HyperLogLogWithSmallSetOptimization
{
private:
using Small = SmallSet<Key, small_set_size>;
using Large = HyperLogLogCounter<K, Hash, DenominatorType>;
Small small;
Large * large = nullptr;
bool isLarge() const
{
return large != nullptr;
}
void toLarge()
{
if (current_memory_tracker)
current_memory_tracker->alloc(sizeof(large));
/// На время копирования данных из tiny, устанавливать значение large ещё нельзя (иначе оно перезатрёт часть данных).
Large * tmp_large = new Large;
for (const auto & x : small)
tmp_large->insert(x);
large = tmp_large;
}
public:
~HyperLogLogWithSmallSetOptimization()
{
if (isLarge())
{
delete large;
if (current_memory_tracker)
current_memory_tracker->free(sizeof(large));
}
}
void insert(Key value)
{
if (!isLarge())
{
if (small.find(value) == small.end())
{
if (!small.full())
small.insert(value);
else
{
toLarge();
large->insert(value);
}
}
}
else
large->insert(value);
}
UInt32 size() const
{
return !isLarge() ? small.size() : large->size();
}
void merge(const HyperLogLogWithSmallSetOptimization & rhs)
{
if (rhs.isLarge())
{
if (!isLarge())
toLarge();
large->merge(*rhs.large);
}
else
{
for (const auto & x : rhs.small)
insert(x);
}
}
/// Можно вызывать только для пустого объекта.
void read(DB::ReadBuffer & in)
{
bool is_large;
readBinary(is_large, in);
if (is_large)
{
toLarge();
large->read(in);
}
else
small.read(in);
}
void readAndMerge(DB::ReadBuffer & in)
{
/// Немного не оптимально.
HyperLogLogWithSmallSetOptimization other;
other.read(in);
merge(other);
}
void write(DB::WriteBuffer & out) const
{
writeBinary(isLarge(), out);
if (isLarge())
large->write(out);
else
small.write(out);
}
};
}

View File

@ -48,4 +48,52 @@ struct UInt128TrivialHash
inline void readBinary(UInt128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void writeBinary(const UInt128 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
/** Используется при агрегации, для укладки большого количества ключей постоянной длины в хэш-таблицу.
*/
struct UInt256
{
UInt64 a;
UInt64 b;
UInt64 c;
UInt64 d;
bool operator== (const UInt256 rhs) const
{
return a == rhs.a && b == rhs.b && c == rhs.c && d == rhs.d;
/* Так получается не лучше.
return 0xFFFF == _mm_movemask_epi8(_mm_and_si128(
_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(&a)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(&rhs.a))),
_mm_cmpeq_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(&c)),
_mm_loadu_si128(reinterpret_cast<const __m128i *>(&rhs.c)))));*/
}
bool operator!= (const UInt256 rhs) const { return !operator==(rhs); }
bool operator== (const UInt64 rhs) const { return a == rhs && b == 0 && c == 0 && d == 0; }
bool operator!= (const UInt64 rhs) const { return !operator==(rhs); }
UInt256 & operator= (const UInt64 rhs) { a = rhs; b = 0; c = 0; d = 0; return *this; }
};
struct UInt256HashCRC32
{
size_t operator()(UInt256 x) const
{
UInt64 crc = -1ULL;
asm("crc32q %[x], %[crc]\n" : [crc] "+r" (crc) : [x] "rm" (x.a));
asm("crc32q %[x], %[crc]\n" : [crc] "+r" (crc) : [x] "rm" (x.b));
asm("crc32q %[x], %[crc]\n" : [crc] "+r" (crc) : [x] "rm" (x.c));
asm("crc32q %[x], %[crc]\n" : [crc] "+r" (crc) : [x] "rm" (x.d));
return crc;
}
};
inline void readBinary(UInt256 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void writeBinary(const UInt256 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
}

View File

@ -42,7 +42,6 @@
#define DEFAULT_MAX_QUERY_SIZE 65536
#define SHOW_CHARS_ON_SYNTAX_ERROR 160L
#define DEFAULT_MAX_THREADS 8
#define DEFAULT_MAX_DISTRIBUTED_CONNECTIONS 1024
#define DEFAULT_INTERACTIVE_DELAY 100000
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024

View File

@ -275,6 +275,7 @@ namespace ErrorCodes
NO_AVAILABLE_REPLICA,
MISMATCH_REPLICAS_DATA_SOURCES,
STORAGE_DOESNT_SUPPORT_PARALLEL_REPLICAS,
CPUID_ERROR,
POCO_EXCEPTION = 1000,
STD_EXCEPTION,

View File

@ -20,8 +20,9 @@ class AggregatingBlockInputStream : public IProfilingBlockInputStream
public:
AggregatingBlockInputStream(BlockInputStreamPtr input_, const ColumnNumbers & keys_, AggregateDescriptions & aggregates_,
bool overflow_row_, bool final_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_,
Compiler * compiler_, UInt32 min_count_to_compile_)
: aggregator(keys_, aggregates_, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_, compiler_, min_count_to_compile_),
Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_)
: aggregator(keys_, aggregates_, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_,
compiler_, min_count_to_compile_, group_by_two_level_threshold_),
final(final_)
{
children.push_back(input_);
@ -33,8 +34,9 @@ public:
*/
AggregatingBlockInputStream(BlockInputStreamPtr input_, const Names & key_names, const AggregateDescriptions & aggregates,
bool overflow_row_, bool final_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_,
Compiler * compiler_, UInt32 min_count_to_compile_)
: aggregator(key_names, aggregates, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_, compiler_, min_count_to_compile_),
Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_)
: aggregator(key_names, aggregates, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_,
compiler_, min_count_to_compile_, group_by_two_level_threshold_),
final(final_)
{
children.push_back(input_);

View File

@ -18,7 +18,7 @@ class MergingAggregatedBlockInputStream : public IProfilingBlockInputStream
public:
MergingAggregatedBlockInputStream(BlockInputStreamPtr input_, const ColumnNumbers & keys_,
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_)
: aggregator(keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0),
: aggregator(keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0),
final(final_), max_threads(max_threads_)
{
children.push_back(input_);
@ -26,7 +26,7 @@ public:
MergingAggregatedBlockInputStream(BlockInputStreamPtr input_, const Names & keys_names_,
const AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_)
: aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0),
: aggregator(keys_names_, aggregates_, overflow_row_, 0, OverflowMode::THROW, nullptr, 0, 0),
final(final_), max_threads(max_threads_)
{
children.push_back(input_);

View File

@ -22,8 +22,9 @@ public:
ParallelAggregatingBlockInputStream(BlockInputStreams inputs, const ColumnNumbers & keys_,
AggregateDescriptions & aggregates_, bool overflow_row_, bool final_, size_t max_threads_,
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_,
Compiler * compiler_, UInt32 min_count_to_compile_)
: aggregator(keys_, aggregates_, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_, compiler_, min_count_to_compile_),
Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_)
: aggregator(keys_, aggregates_, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_,
compiler_, min_count_to_compile_, group_by_two_level_threshold_),
final(final_), max_threads(std::min(inputs.size(), max_threads_)),
keys_size(keys_.size()), aggregates_size(aggregates_.size()),
handler(*this), processor(inputs, max_threads, handler)
@ -36,8 +37,9 @@ public:
ParallelAggregatingBlockInputStream(BlockInputStreams inputs, const Names & key_names,
const AggregateDescriptions & aggregates, bool overflow_row_, bool final_, size_t max_threads_,
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_,
Compiler * compiler_, UInt32 min_count_to_compile_)
: aggregator(key_names, aggregates, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_, compiler_, min_count_to_compile_),
Compiler * compiler_, UInt32 min_count_to_compile_, size_t group_by_two_level_threshold_)
: aggregator(key_names, aggregates, overflow_row_, max_rows_to_group_by_, group_by_overflow_mode_,
compiler_, min_count_to_compile_, group_by_two_level_threshold_),
final(final_), max_threads(std::min(inputs.size(), max_threads_)),
keys_size(key_names.size()), aggregates_size(aggregates.size()),
handler(*this), processor(inputs, max_threads, handler)
@ -129,7 +131,8 @@ private:
{
parent.aggregator.executeOnBlock(block, *parent.many_data[thread_num],
parent.threads_data[thread_num].key_columns, parent.threads_data[thread_num].aggregate_columns,
parent.threads_data[thread_num].key_sizes, parent.threads_data[thread_num].key, parent.no_more_keys);
parent.threads_data[thread_num].key_sizes, parent.threads_data[thread_num].key,
parent.no_more_keys);
parent.threads_data[thread_num].src_rows += block.rowsInFirstColumn();
parent.threads_data[thread_num].src_bytes += block.bytes();

View File

@ -22,6 +22,12 @@ class RemoteBlockInputStream : public IProfilingBlockInputStream
private:
void init(const Settings * settings_)
{
sent_query.store(false, std::memory_order_seq_cst);
finished.store(false, std::memory_order_seq_cst);
got_exception_from_replica.store(false, std::memory_order_seq_cst);
got_unknown_packet_from_replica.store(false, std::memory_order_seq_cst);
was_cancelled.store(false, std::memory_order_seq_cst);
if (settings_)
{
send_settings = true;
@ -83,14 +89,13 @@ public:
if (!__sync_bool_compare_and_swap(&is_cancelled, false, true))
return;
if (isQueryInProgress() && !hasThrownException())
if (hasNoQueryInProgress() || hasThrownException())
return;
if (tryCancel())
{
std::string addresses = parallel_replicas->dumpAddresses();
LOG_TRACE(log, "(" + addresses + ") Cancelling query");
/// Если запрошено прервать запрос - попросим удалённые реплики тоже прервать запрос.
was_cancelled = true;
parallel_replicas->sendCancel();
}
}
@ -101,7 +106,7 @@ public:
* все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы
* эти соединения не остались висеть в рассихронизированном состоянии.
*/
if (isQueryInProgress())
if (established || isQueryInProgress())
parallel_replicas->disconnect();
}
@ -136,13 +141,14 @@ protected:
Block readImpl() override
{
if (!sent_query)
if (!sent_query.load(std::memory_order_seq_cst))
{
createParallelReplicas();
established = true;
parallel_replicas->sendQuery(query, "", stage, true);
established = false;
sent_query.store(true, std::memory_order_seq_cst);
sendExternalTables();
__sync_synchronize();
sent_query = true;
}
while (true)
@ -158,14 +164,14 @@ protected:
break; /// Если блок пустой - получим другие пакеты до EndOfStream.
case Protocol::Server::Exception:
got_exception_from_replica = true;
got_exception_from_replica.store(true, std::memory_order_seq_cst);
packet.exception->rethrow();
break;
case Protocol::Server::EndOfStream:
if (!parallel_replicas->hasActiveReplicas())
{
finished = true;
finished.store(true, std::memory_order_seq_cst);
return Block();
}
break;
@ -197,7 +203,7 @@ protected:
break;
default:
got_unknown_packet_from_replica = true;
got_unknown_packet_from_replica.store(true, std::memory_order_seq_cst);
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
}
@ -220,13 +226,10 @@ protected:
*/
/// Отправим просьбу прервать выполнение запроса, если ещё не отправляли.
if (!was_cancelled)
if (tryCancel())
{
std::string addresses = parallel_replicas->dumpAddresses();
LOG_TRACE(log, "(" + addresses + ") Cancelling query because enough data has been read");
was_cancelled = true;
parallel_replicas->sendCancel();
}
/// Получим оставшиеся пакеты, чтобы не было рассинхронизации в соединениях с репликами.
@ -234,16 +237,16 @@ protected:
switch (packet.type)
{
case Protocol::Server::EndOfStream:
finished = true;
finished.store(true, std::memory_order_seq_cst);
break;
case Protocol::Server::Exception:
got_exception_from_replica = true;
got_exception_from_replica.store(true, std::memory_order_seq_cst);
packet.exception->rethrow();
break;
default:
got_unknown_packet_from_replica = true;
got_unknown_packet_from_replica.store(true, std::memory_order_seq_cst);
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
}
@ -261,19 +264,41 @@ protected:
/// Возвращает true, если запрос отправлен, а ещё не выполнен.
bool isQueryInProgress() const
{
return sent_query && !finished && !was_cancelled;
return sent_query.load(std::memory_order_seq_cst) && !finished.load(std::memory_order_seq_cst) && !was_cancelled.load(std::memory_order_seq_cst);
}
/// Возвращает true, если никакой запрос не отправлен или один запрос уже выполнен.
bool hasNoQueryInProgress() const
{
return !sent_query || finished;
return !sent_query.load(std::memory_order_seq_cst) || finished.load(std::memory_order_seq_cst);
}
/// Возвращает true, если исключение было выкинуто.
bool hasThrownException() const
{
return got_exception_from_replica || got_unknown_packet_from_replica;
return got_exception_from_replica.load(std::memory_order_seq_cst) || got_unknown_packet_from_replica.load(std::memory_order_seq_cst);
}
private:
/// ITable::read requires a Context, therefore we should create one if the user can't supply it
static Context & getDefaultContext()
{
static Context instance;
return instance;
}
/// Отправить запрос на отмену всех соединений к репликам, если такой запрос ещё не был отправлен.
bool tryCancel()
{
bool old_val = false;
bool new_val = true;
if (was_cancelled.compare_exchange_strong(old_val, new_val, std::memory_order_seq_cst, std::memory_order_relaxed))
{
parallel_replicas->sendCancel();
return true;
}
else
return false;
}
private:
@ -293,40 +318,36 @@ private:
QueryProcessingStage::Enum stage;
Context context;
/// Установили соединения с репликами, но ещё не отправили запрос.
volatile bool established = false;
/// Отправили запрос (это делается перед получением первого блока).
bool sent_query = false;
std::atomic<bool> sent_query;
/** Получили все данные от всех реплик, до пакета EndOfStream.
* Если при уничтожении объекта, ещё не все данные считаны,
* то для того, чтобы не было рассинхронизации, на реплики отправляются просьбы прервать выполнение запроса,
* и после этого считываются все пакеты до EndOfStream.
*/
bool finished = false;
std::atomic<bool> finished;
/** На каждую реплику была отправлена просьба прервать выполнение запроса, так как данные больше не нужны.
* Это может быть из-за того, что данных достаточно (например, при использовании LIMIT),
* или если на стороне клиента произошло исключение.
*/
bool was_cancelled = false;
std::atomic<bool> was_cancelled;
/** С одной репилки было получено исключение. В этом случае получать больше пакетов или
* просить прервать запрос на этой реплике не нужно.
*/
bool got_exception_from_replica = false;
std::atomic<bool> got_exception_from_replica;
/** С одной реплики был получен неизвестный пакет. В этом случае получать больше пакетов или
* просить прервать запрос на этой реплике не нужно.
*/
bool got_unknown_packet_from_replica = false;
std::atomic<bool> got_unknown_packet_from_replica;
Logger * log = &Logger::get("RemoteBlockInputStream");
/// ITable::read requires a Context, therefore we should create one if the user can't supply it
static Context & getDefaultContext()
{
static Context instance;
return instance;
}
};
}

View File

@ -1237,7 +1237,7 @@ private:
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
{
res_values[j] = ++indices[pack128(j, count, columns, key_sizes)];
res_values[j] = ++indices[packFixed<UInt128>(j, count, columns, key_sizes)];
}
prev_off = off;
}

View File

@ -9,6 +9,8 @@
#include <DB/Core/Defines.h>
#include <DB/Core/StringRef.h>
#include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnFixedString.h>
template <>
@ -21,23 +23,43 @@ namespace DB
typedef std::vector<size_t> Sizes;
/// Записать набор ключей фиксированной длины в UInt128, уложив их подряд (при допущении, что они помещаются).
static inline UInt128 ALWAYS_INLINE pack128(
/// Записать набор ключей фиксированной длины в T, уложив их подряд (при допущении, что они помещаются).
template <typename T>
static inline T ALWAYS_INLINE packFixed(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, const Sizes & key_sizes)
{
union
{
UInt128 key;
char bytes[16];
T key;
char bytes[sizeof(key)];
};
memset(bytes, 0, 16);
memset(bytes, 0, sizeof(key));
size_t offset = 0;
for (size_t j = 0; j < keys_size; ++j)
{
StringRef key_data = key_columns[j]->getDataAt(i);
memcpy(bytes + offset, key_data.data, key_sizes[j]);
offset += key_sizes[j];
switch (key_sizes[j])
{
case 1:
memcpy(bytes + offset, &static_cast<const ColumnUInt8 *>(key_columns[j])->getData()[i], 1);
offset += 1;
break;
case 2:
memcpy(bytes + offset, &static_cast<const ColumnUInt16 *>(key_columns[j])->getData()[i], 2);
offset += 2;
break;
case 4:
memcpy(bytes + offset, &static_cast<const ColumnUInt32 *>(key_columns[j])->getData()[i], 4);
offset += 4;
break;
case 8:
memcpy(bytes + offset, &static_cast<const ColumnUInt64 *>(key_columns[j])->getData()[i], 8);
offset += 8;
break;
default:
memcpy(bytes + offset, &static_cast<const ColumnFixedString *>(key_columns[j])->getChars()[i * key_sizes[j]], key_sizes[j]);
offset += key_sizes[j];
}
}
return key;
@ -123,4 +145,41 @@ static inline StringRef * ALWAYS_INLINE extractKeysAndPlaceInPool(
}
/** Скопировать ключи в пул в непрерывный кусок памяти.
* Потом разместить в пуле StringRef-ы на них.
*
* [key1][key2]...[keyN][ref1][ref2]...[refN]
* ^---------------------| |
* ^---------------------|
* ^---return-value----^
*
* Вернуть StringRef на кусок памяти с ключами (без учёта StringRef-ов после них).
*/
static inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
{
size_t sum_keys_size = 0;
for (size_t j = 0; j < keys_size; ++j)
{
keys[j] = key_columns[j]->getDataAtWithTerminatingZero(i);
sum_keys_size += keys[j].size;
}
char * res = pool.alloc(sum_keys_size + keys_size * sizeof(StringRef));
char * place = res;
for (size_t j = 0; j < keys_size; ++j)
{
memcpy(place, keys[j].data, keys[j].size);
keys[j].data = place;
place += keys[j].size;
}
/// Размещаем в пуле StringRef-ы на только что скопированные ключи.
memcpy(place, &keys[0], keys_size * sizeof(StringRef));
return {res, sum_keys_size};
}
}

View File

@ -52,11 +52,13 @@ typedef AggregateDataPtr AggregatedDataWithoutKey;
typedef HashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>> AggregatedDataWithUInt64Key;
typedef HashMapWithSavedHash<StringRef, AggregateDataPtr> AggregatedDataWithStringKey;
typedef HashMap<UInt128, AggregateDataPtr, UInt128HashCRC32> AggregatedDataWithKeys128;
typedef HashMap<UInt256, AggregateDataPtr, UInt256HashCRC32> AggregatedDataWithKeys256;
typedef HashMap<UInt128, std::pair<StringRef*, AggregateDataPtr>, UInt128TrivialHash> AggregatedDataHashed;
typedef TwoLevelHashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>> AggregatedDataWithUInt64KeyTwoLevel;
typedef TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr> AggregatedDataWithStringKeyTwoLevel;
typedef TwoLevelHashMap<UInt128, AggregateDataPtr, UInt128HashCRC32> AggregatedDataWithKeys128TwoLevel;
typedef TwoLevelHashMap<UInt256, AggregateDataPtr, UInt256HashCRC32> AggregatedDataWithKeys256TwoLevel;
typedef TwoLevelHashMap<UInt128, std::pair<StringRef*, AggregateDataPtr>, UInt128TrivialHash> AggregatedDataHashedTwoLevel;
typedef HashMap<UInt64, AggregateDataPtr, TrivialHash, HashTableFixedGrower<8>> AggregatedDataWithUInt8Key;
@ -99,7 +101,8 @@ struct AggregationMethodOneNumber
size_t keys_size, /// Количество ключевых столбцов.
size_t i, /// Из какой строки блока достать ключ.
const Sizes & key_sizes, /// Если ключи фиксированной длины - их длины. Не используется в методах агрегации по ключам переменной длины.
StringRefs & keys) const /// Сюда могут быть записаны ссылки на данные ключей в столбцах. Они могут быть использованы в дальнейшем.
StringRefs & keys, /// Сюда могут быть записаны ссылки на данные ключей в столбцах. Они могут быть использованы в дальнейшем.
Arena & pool) const
{
return unionCastToUInt64(vec[i]);
}
@ -115,6 +118,14 @@ struct AggregationMethodOneNumber
{
}
/** Действие, которое нужно сделать, если ключ не новый. Например, откатить выделение памяти в пуле.
*/
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
/** Не использовать оптимизацию для идущих подряд ключей.
*/
static const bool no_consecutive_keys_optimization = false;
/** Вставить ключ из хэш-таблицы в столбцы.
*/
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
@ -159,7 +170,8 @@ struct AggregationMethodString
size_t keys_size,
size_t i,
const Sizes & key_sizes,
StringRefs & keys) const
StringRefs & keys,
Arena & pool) const
{
return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]],
@ -175,6 +187,10 @@ struct AggregationMethodString
value.first.data = pool.insert(value.first.data, value.first.size);
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static const bool no_consecutive_keys_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
key_columns[0]->insertData(value.first.data, value.first.size);
@ -217,7 +233,8 @@ struct AggregationMethodFixedString
size_t keys_size,
size_t i,
const Sizes & key_sizes,
StringRefs & keys) const
StringRefs & keys,
Arena & pool) const
{
return StringRef(&(*chars)[i * n], n);
}
@ -231,6 +248,10 @@ struct AggregationMethodFixedString
value.first.data = pool.insert(value.first.data, value.first.size);
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static const bool no_consecutive_keys_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
key_columns[0]->insertData(value.first.data, value.first.size);
@ -238,9 +259,9 @@ struct AggregationMethodFixedString
};
/// Для случая, когда все ключи фиксированной длины, и они помещаются в 128 бит.
/// Для случая, когда все ключи фиксированной длины, и они помещаются в N (например, 128) бит.
template <typename TData>
struct AggregationMethodKeys128
struct AggregationMethodKeysFixed
{
typedef TData Data;
typedef typename Data::key_type Key;
@ -250,10 +271,10 @@ struct AggregationMethodKeys128
Data data;
AggregationMethodKeys128() {}
AggregationMethodKeysFixed() {}
template <typename Other>
AggregationMethodKeys128(const Other & other) : data(other.data) {}
AggregationMethodKeysFixed(const Other & other) : data(other.data) {}
struct State
{
@ -266,9 +287,10 @@ struct AggregationMethodKeys128
size_t keys_size,
size_t i,
const Sizes & key_sizes,
StringRefs & keys) const
StringRefs & keys,
Arena & pool) const
{
return pack128(i, keys_size, key_columns, key_sizes);
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
};
@ -279,6 +301,10 @@ struct AggregationMethodKeys128
{
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static const bool no_consecutive_keys_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
size_t offset = 0;
@ -292,6 +318,67 @@ struct AggregationMethodKeys128
};
/// Для остальных случаев. Агрегирует по конкатенации ключей. (При этом, строки, содержащие нули посередине, могут склеиться.)
template <typename TData>
struct AggregationMethodConcat
{
typedef TData Data;
typedef typename Data::key_type Key;
typedef typename Data::mapped_type Mapped;
typedef typename Data::iterator iterator;
typedef typename Data::const_iterator const_iterator;
Data data;
AggregationMethodConcat() {}
template <typename Other>
AggregationMethodConcat(const Other & other) : data(other.data) {}
struct State
{
void init(ConstColumnPlainPtrs & key_columns)
{
}
Key getKey(
const ConstColumnPlainPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes,
StringRefs & keys,
Arena & pool) const
{
return extractKeysAndPlaceInPoolContiguous(i, keys_size, key_columns, keys, pool);
}
};
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
{
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool)
{
pool.rollback(key.size + keys.size() * sizeof(keys[0]));
}
/// Если ключ уже был, то он удаляется из пула (затирается), и сравнить с ним следующий ключ уже нельзя.
static const bool no_consecutive_keys_optimization = true;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
/// См. функцию extractKeysAndPlaceInPoolContiguous.
const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size);
for (size_t i = 0; i < keys_size; ++i)
key_columns[i]->insertDataWithTerminatingZero(key_refs[i].data, key_refs[i].size);
}
};
/// Для остальных случаев. Агрегирует по 128 битному хэшу от ключа. (При этом, строки, содержащие нули посередине, могут склеиться.)
template <typename TData>
struct AggregationMethodHashed
@ -320,7 +407,8 @@ struct AggregationMethodHashed
size_t keys_size,
size_t i,
const Sizes & key_sizes,
StringRefs & keys) const
StringRefs & keys,
Arena & pool) const
{
return hash128(i, keys_size, key_columns, keys);
}
@ -334,6 +422,10 @@ struct AggregationMethodHashed
value.second.first = placeKeysInPool(i, keys_size, keys, pool);
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) {}
static const bool no_consecutive_keys_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
for (size_t i = 0; i < keys_size; ++i)
@ -383,15 +475,19 @@ struct AggregatedDataVariants : private boost::noncopyable
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64Key>> key64;
std::unique_ptr<AggregationMethodString<AggregatedDataWithStringKey>> key_string;
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKey>> key_fixed_string;
std::unique_ptr<AggregationMethodKeys128<AggregatedDataWithKeys128>> keys128;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128>> keys128;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256>> keys256;
std::unique_ptr<AggregationMethodHashed<AggregatedDataHashed>> hashed;
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKey>> concat;
std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64KeyTwoLevel>> key32_two_level;
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64KeyTwoLevel>> key64_two_level;
std::unique_ptr<AggregationMethodString<AggregatedDataWithStringKeyTwoLevel>> key_string_two_level;
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKeyTwoLevel>> key_fixed_string_two_level;
std::unique_ptr<AggregationMethodKeys128<AggregatedDataWithKeys128TwoLevel>> keys128_two_level;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128TwoLevel>> keys128_two_level;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel>> keys256_two_level;
std::unique_ptr<AggregationMethodHashed<AggregatedDataHashedTwoLevel>> hashed_two_level;
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKeyTwoLevel>> concat_two_level;
/// В этом и подобных макросах, вариант without_key не учитывается.
#define APPLY_FOR_AGGREGATED_VARIANTS(M) \
@ -402,13 +498,17 @@ struct AggregatedDataVariants : private boost::noncopyable
M(key_string, false) \
M(key_fixed_string, false) \
M(keys128, false) \
M(keys256, false) \
M(hashed, false) \
M(concat, false) \
M(key32_two_level, true) \
M(key64_two_level, true) \
M(key_string_two_level, true) \
M(key_fixed_string_two_level, true) \
M(keys128_two_level, true) \
M(hashed_two_level, true)
M(keys256_two_level, true) \
M(hashed_two_level, true) \
M(concat_two_level, true)
enum class Type
{
@ -520,7 +620,9 @@ struct AggregatedDataVariants : private boost::noncopyable
M(key_string) \
M(key_fixed_string) \
M(keys128) \
M(hashed)
M(keys256) \
M(hashed) \
M(concat)
#define APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) \
M(key8) \
@ -549,7 +651,9 @@ struct AggregatedDataVariants : private boost::noncopyable
M(key_string_two_level) \
M(key_fixed_string_two_level) \
M(keys128_two_level) \
M(hashed_two_level)
M(keys256_two_level) \
M(hashed_two_level) \
M(concat_two_level)
};
typedef SharedPtr<AggregatedDataVariants> AggregatedDataVariantsPtr;
@ -573,11 +677,12 @@ class Aggregator
{
public:
Aggregator(const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_,
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, Compiler * compiler_, UInt32 min_count_to_compile_)
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, Compiler * compiler_, UInt32 min_count_to_compile_,
size_t group_by_two_level_threshold_)
: keys(keys_), aggregates(aggregates_), aggregates_size(aggregates.size()),
overflow_row(overflow_row_),
max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_),
compiler(compiler_), min_count_to_compile(min_count_to_compile_)
compiler(compiler_), min_count_to_compile(min_count_to_compile_), group_by_two_level_threshold(group_by_two_level_threshold_)
{
std::sort(keys.begin(), keys.end());
keys.erase(std::unique(keys.begin(), keys.end()), keys.end());
@ -585,11 +690,12 @@ public:
}
Aggregator(const Names & key_names_, const AggregateDescriptions & aggregates_, bool overflow_row_,
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, Compiler * compiler_, UInt32 min_count_to_compile_)
size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, Compiler * compiler_, UInt32 min_count_to_compile_,
size_t group_by_two_level_threshold_)
: key_names(key_names_), aggregates(aggregates_), aggregates_size(aggregates.size()),
overflow_row(overflow_row_),
max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_),
compiler(compiler_), min_count_to_compile(min_count_to_compile_)
compiler(compiler_), min_count_to_compile(min_count_to_compile_), group_by_two_level_threshold(group_by_two_level_threshold_)
{
std::sort(key_names.begin(), key_names.end());
key_names.erase(std::unique(key_names.begin(), key_names.end()), key_names.end());
@ -685,6 +791,10 @@ protected:
bool compiled_if_possible = false;
void compileIfPossible(AggregatedDataVariants::Type type);
/** При каком количестве ключей, начинает использоваться двухуровневая агрегация.
* 0 - никогда не использовать.
*/
size_t group_by_two_level_threshold;
/** Если заданы только имена столбцов (key_names, а также aggregates[i].column_name), то вычислить номера столбцов.
* Сформировать блок - пример результата.

View File

@ -35,8 +35,8 @@ struct Settings
M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE) \
/** Максимальный размер блока для вставки, если мы управляем формированием блоков для вставки. */ \
M(SettingUInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE) \
/** Максимальное количество потоков выполнения запроса */ \
M(SettingUInt64, max_threads, DEFAULT_MAX_THREADS) \
/** Максимальное количество потоков выполнения запроса. По-умолчанию - определять автоматически. */ \
M(SettingMaxThreads, max_threads, 0) \
/** Максимальное количество соединений при распределённой обработке одного запроса (должно быть больше, чем max_threads). */ \
M(SettingUInt64, max_distributed_connections, DEFAULT_MAX_DISTRIBUTED_CONNECTIONS) \
/** Какую часть запроса можно прочитать в оперативку для парсинга (оставшиеся данные для INSERT, если есть, считываются позже) */ \
@ -87,6 +87,8 @@ struct Settings
M(SettingBool, compile, false) \
/** Количество одинаковых по структуре запросов перед тем, как инициируется их компиляция. */ \
M(SettingUInt64, min_count_to_compile, 0) \
/** При каком количестве ключей, начинает использоваться двухуровневая агрегация. 0 - никогда не использовать. */ \
M(SettingUInt64, group_by_two_level_threshold, 30000) \
\
/** Максимальное количество используемых реплик каждого шарда при выполнении запроса */ \
M(SettingUInt64, max_parallel_replicas, 1) \
@ -106,7 +108,7 @@ struct Settings
M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \
\
/** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \
M(SettingUInt64, min_or_chain_length_for_optimization, 6) \
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 4) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

@ -3,6 +3,7 @@
#include <DB/Core/Field.h>
#include <DB/IO/WriteHelpers.h>
#include <Poco/Timespan.h>
#include <cpuid/libcpuid.h>
namespace DB
@ -64,6 +65,90 @@ struct SettingUInt64
typedef SettingUInt64 SettingBool;
/** В отличие от SettingUInt64, поддерживает значение 'auto' - количество процессорных ядер без учёта SMT.
* Значение 0 так же воспринимается как auto.
* При сериализации, auto записывается так же, как 0.
*/
struct SettingMaxThreads
{
UInt64 value;
bool is_auto;
bool changed = false;
SettingMaxThreads(UInt64 x = 0) : value(x ? x : getAutoValue()), is_auto(x == 0) {}
operator UInt64() const { return value; }
SettingMaxThreads & operator= (UInt64 x) { set(x); return *this; }
String toString() const
{
/// Вместо значения auto выводим актуальное значение, чтобы его было легче посмотреть.
return DB::toString(value);
}
void set(UInt64 x)
{
value = x ? x : getAutoValue();
is_auto = x == 0;
changed = true;
}
void set(const Field & x)
{
if (x.getType() == Field::Types::String)
set(safeGet<const String &>(x));
else
set(safeGet<UInt64>(x));
}
void set(const String & x)
{
if (x == "auto")
setAuto();
else
set(parse<UInt64>(x));
}
void set(ReadBuffer & buf)
{
UInt64 x = 0;
readVarUInt(x, buf);
set(x);
}
void write(WriteBuffer & buf) const
{
writeVarUInt(is_auto ? 0 : value, buf);
}
void setAuto()
{
value = getAutoValue();
is_auto = true;
}
UInt64 getAutoValue() const
{
static auto res = getAutoValueImpl();
return res;
}
/// Выполняется один раз за всё время. Выполняется из одного потока.
UInt64 getAutoValueImpl() const
{
cpu_raw_data_t raw_data;
if (0 != cpuid_get_raw_data(&raw_data))
throw Exception("Cannot cpuid_get_raw_data: " + String(cpuid_error()), ErrorCodes::CPUID_ERROR);
cpu_id_t data;
if (0 != cpu_identify(&raw_data, &data))
throw Exception("Cannot cpu_identify: " + String(cpuid_error()), ErrorCodes::CPUID_ERROR);
return data.num_cores * data.total_logical_cpus / data.num_logical_cpus;
}
};
struct SettingSeconds
{
Poco::Timespan value;

View File

@ -193,23 +193,27 @@ void NO_INLINE Aggregator::executeSpecializedCase(
bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys.
/// Получаем ключ для вставки в хэш-таблицу.
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes, keys);
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes, keys, *aggregates_pool);
if (!no_more_keys) /// Вставляем.
{
/// Оптимизация для часто повторяющихся ключей.
if (i != 0 && key == prev_key)
if (!Method::no_consecutive_keys_optimization)
{
AggregateDataPtr value = Method::getAggregateData(it->second);
if (i != 0 && key == prev_key)
{
AggregateDataPtr value = Method::getAggregateData(it->second);
/// Добавляем значения в агрегатные функции.
AggregateFunctionsList::forEach(AggregateFunctionsUpdater(
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, value, i));
/// Добавляем значения в агрегатные функции.
AggregateFunctionsList::forEach(AggregateFunctionsUpdater(
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, value, i));
continue;
method.onExistingKey(key, keys, *aggregates_pool);
continue;
}
else
prev_key = key;
}
else
prev_key = key;
method.data.emplace(key, it, inserted);
}
@ -224,7 +228,10 @@ void NO_INLINE Aggregator::executeSpecializedCase(
/// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего.
if (no_more_keys && overflow && !overflow_row)
{
method.onExistingKey(key, keys, *aggregates_pool);
continue;
}
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
if (inserted)
@ -237,6 +244,8 @@ void NO_INLINE Aggregator::executeSpecializedCase(
AggregateFunctionsList::forEach(AggregateFunctionsCreator(
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, aggregate_data));
}
else
method.onExistingKey(key, keys, *aggregates_pool);
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;

View File

@ -6,6 +6,10 @@
#include <DB/Core/Block.h>
#include <DB/Storages/ColumnDefault.h>
#include <boost/range/iterator_range.hpp>
#include <boost/range/join.hpp>
namespace DB
{
@ -94,6 +98,11 @@ public:
private:
virtual const NamesAndTypesList & getColumnsListImpl() const = 0;
using ColumnsListRange = boost::range::joined_range<
const boost::iterator_range<NamesAndTypesList::const_iterator>,
const boost::iterator_range<NamesAndTypesList::const_iterator>>;
ColumnsListRange getColumnsListIterator() const;
};
}

View File

@ -210,11 +210,11 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqHLL12Data>;
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqHLL12Data<DataTypeDate::FieldType>>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqHLL12Data>;
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqHLL12Data<DataTypeDateTime::FieldType>>;
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniq<String, AggregateFunctionUniqHLL12Data>;
return new AggregateFunctionUniq<String, AggregateFunctionUniqHLL12Data<String>>;
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -43,6 +43,8 @@ ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_
void ParallelReplicas::sendExternalTablesData(std::vector<ExternalTablesData> & data)
{
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
if (!sent_query)
throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
@ -61,6 +63,8 @@ void ParallelReplicas::sendExternalTablesData(std::vector<ExternalTablesData> &
void ParallelReplicas::sendQuery(const String & query, const String & query_id, UInt64 stage, bool with_pending_data)
{
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
if (sent_query)
throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR);
@ -97,6 +101,110 @@ void ParallelReplicas::sendQuery(const String & query, const String & query_id,
}
Connection::Packet ParallelReplicas::receivePacket()
{
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
return receivePacketUnlocked();
}
void ParallelReplicas::disconnect()
{
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
for (auto it = replica_map.begin(); it != replica_map.end(); ++it)
{
Connection * connection = it->second;
if (connection != nullptr)
{
connection->disconnect();
invalidateReplica(it);
}
}
}
void ParallelReplicas::sendCancel()
{
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
if (!sent_query || cancelled)
throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR);
for (auto & e : replica_map)
{
Connection * connection = e.second;
if (connection != nullptr)
connection->sendCancel();
}
cancelled = true;
}
Connection::Packet ParallelReplicas::drain()
{
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
if (!cancelled)
throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR);
Connection::Packet res;
res.type = Protocol::Server::EndOfStream;
while (hasActiveReplicas())
{
Connection::Packet packet = receivePacketUnlocked();
switch (packet.type)
{
case Protocol::Server::Data:
case Protocol::Server::Progress:
case Protocol::Server::ProfileInfo:
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::EndOfStream:
break;
case Protocol::Server::Exception:
default:
/// Если мы получили исключение или неизвестный пакет, сохраняем его.
res = packet;
break;
}
}
return res;
}
std::string ParallelReplicas::dumpAddresses() const
{
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
bool is_first = true;
std::ostringstream os;
for (auto & e : replica_map)
{
const Connection * connection = e.second;
if (connection != nullptr)
{
os << (is_first ? "" : "; ") << connection->getServerAddress();
if (is_first) { is_first = false; }
}
}
return os.str();
}
void ParallelReplicas::registerReplica(Connection * connection)
{
if (connection == nullptr)
throw Exception("Invalid connection specified in parameter.", ErrorCodes::LOGICAL_ERROR);
auto res = replica_map.insert(std::make_pair(connection->socket.impl()->sockfd(), connection));
if (!res.second)
throw Exception("Invalid set of connections.", ErrorCodes::LOGICAL_ERROR);
if (throttler)
connection->setThrottler(throttler);
}
Connection::Packet ParallelReplicas::receivePacketUnlocked()
{
if (!sent_query)
throw Exception("Cannot receive packets: no query sent.", ErrorCodes::LOGICAL_ERROR);
@ -133,98 +241,6 @@ Connection::Packet ParallelReplicas::receivePacket()
return packet;
}
void ParallelReplicas::disconnect()
{
for (auto it = replica_map.begin(); it != replica_map.end(); ++it)
{
Connection * connection = it->second;
if (connection != nullptr)
{
connection->disconnect();
invalidateReplica(it);
}
}
}
void ParallelReplicas::sendCancel()
{
if (!sent_query || cancelled)
throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR);
for (auto & e : replica_map)
{
Connection * connection = e.second;
if (connection != nullptr)
connection->sendCancel();
}
cancelled = true;
}
Connection::Packet ParallelReplicas::drain()
{
if (!cancelled)
throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR);
Connection::Packet res;
res.type = Protocol::Server::EndOfStream;
while (hasActiveReplicas())
{
Connection::Packet packet = receivePacket();
switch (packet.type)
{
case Protocol::Server::Data:
case Protocol::Server::Progress:
case Protocol::Server::ProfileInfo:
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::EndOfStream:
break;
case Protocol::Server::Exception:
default:
/// Если мы получили исключение или неизвестный пакет, сохраняем его.
res = packet;
break;
}
}
return res;
}
std::string ParallelReplicas::dumpAddresses() const
{
bool is_first = true;
std::ostringstream os;
for (auto & e : replica_map)
{
const Connection * connection = e.second;
if (connection != nullptr)
{
os << (is_first ? "" : "; ") << connection->getServerAddress();
if (is_first) { is_first = false; }
}
}
return os.str();
}
void ParallelReplicas::registerReplica(Connection * connection)
{
if (connection == nullptr)
throw Exception("Invalid connection specified in parameter.", ErrorCodes::LOGICAL_ERROR);
auto res = replica_map.insert(std::make_pair(connection->socket.impl()->sockfd(), connection));
if (!res.second)
throw Exception("Invalid set of connections.", ErrorCodes::LOGICAL_ERROR);
if (throttler)
connection->setThrottler(throttler);
}
ParallelReplicas::ReplicaMap::iterator ParallelReplicas::getReplicaForReading()
{
ReplicaMap::iterator it;

View File

@ -34,7 +34,7 @@ int main(int argc, char ** argv)
column_x.type = new DB::DataTypeInt16;
DB::ColumnInt16 * x = new DB::ColumnInt16;
column_x.column = x;
DB::PODArray<Int16> & vec_x = x->getData();
auto & vec_x = x->getData();
vec_x.resize(n);
for (size_t i = 0; i < n; ++i)
@ -92,7 +92,7 @@ int main(int argc, char ** argv)
DB::BlockInputStreamPtr stream = new DB::OneBlockInputStream(block);
stream = new DB::AggregatingBlockInputStream(stream, key_column_numbers, aggregate_descriptions, false, true,
0, DB::OverflowMode::THROW, nullptr, 0);
0, DB::OverflowMode::THROW, nullptr, 0, 0);
DB::WriteBufferFromOStream ob(std::cout);
DB::RowOutputStreamPtr row_out = new DB::TabSeparatedRowOutputStream(ob, sample);

View File

@ -311,21 +311,19 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod(const ConstColu
* Затем, в процессе работы, данные могут быть переконвертированы в two-level структуру, если их становится много.
*/
bool keys_fit_128_bits = true;
bool all_fixed = true;
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j)
{
if (!key_columns[j]->isFixed())
{
keys_fit_128_bits = false;
all_fixed = false;
break;
}
key_sizes[j] = key_columns[j]->sizeOfField();
keys_bytes += key_sizes[j];
}
if (keys_bytes > 16)
keys_fit_128_bits = false;
/// Если ключей нет
if (keys_size == 0)
@ -346,9 +344,11 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod(const ConstColu
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8.", ErrorCodes::LOGICAL_ERROR);
}
/// Если ключи помещаются в 128 бит, будем использовать хэш-таблицу по упакованным в 128-бит ключам
if (keys_fit_128_bits)
/// Если ключи помещаются в N бит, будем использовать хэш-таблицу по упакованным в N-бит ключам
if (all_fixed && keys_bytes <= 16)
return AggregatedDataVariants::Type::keys128;
if (all_fixed && keys_bytes <= 32)
return AggregatedDataVariants::Type::keys256;
/// Если есть один строковый ключ, то используем хэш-таблицу с ним
if (keys_size == 1 && typeid_cast<const ColumnString *>(key_columns[0]))
@ -357,8 +357,10 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod(const ConstColu
if (keys_size == 1 && typeid_cast<const ColumnFixedString *>(key_columns[0]))
return AggregatedDataVariants::Type::key_fixed_string;
/// Иначе будем агрегировать по хэшу от ключей.
return AggregatedDataVariants::Type::hashed;
/// Иначе будем агрегировать по конкатенации ключей.
return AggregatedDataVariants::Type::concat;
/// NOTE AggregatedDataVariants::Type::hashed не используется.
}
@ -439,22 +441,26 @@ void NO_INLINE Aggregator::executeImplCase(
bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys.
/// Получаем ключ для вставки в хэш-таблицу.
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes, keys);
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes, keys, *aggregates_pool);
if (!no_more_keys) /// Вставляем.
{
/// Оптимизация для часто повторяющихся ключей.
if (i != 0 && key == prev_key)
if (!Method::no_consecutive_keys_optimization)
{
/// Добавляем значения в агрегатные функции.
AggregateDataPtr value = Method::getAggregateData(it->second);
for (size_t j = 0; j < aggregates_size; ++j) /// NOTE: Заменить индекс на два указателя?
aggregate_functions[j]->add(value + offsets_of_aggregate_states[j], &aggregate_columns[j][0], i);
if (i != 0 && key == prev_key)
{
/// Добавляем значения в агрегатные функции.
AggregateDataPtr value = Method::getAggregateData(it->second);
for (size_t j = 0; j < aggregates_size; ++j) /// NOTE: Заменить индекс на два указателя?
aggregate_functions[j]->add(value + offsets_of_aggregate_states[j], &aggregate_columns[j][0], i);
continue;
method.onExistingKey(key, keys, *aggregates_pool);
continue;
}
else
prev_key = key;
}
else
prev_key = key;
method.data.emplace(key, it, inserted);
}
@ -469,7 +475,10 @@ void NO_INLINE Aggregator::executeImplCase(
/// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего.
if (no_more_keys && overflow && !overflow_row)
{
method.onExistingKey(key, keys, *aggregates_pool);
continue;
}
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
if (inserted)
@ -480,6 +489,8 @@ void NO_INLINE Aggregator::executeImplCase(
aggregate_data = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(aggregate_data);
}
else
method.onExistingKey(key, keys, *aggregates_pool);
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
@ -644,16 +655,7 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
size_t result_size = result.sizeWithoutOverflowRow();
/// Если результат уже достаточно большой, и его можно сконвертировать в двухуровневую хэш-таблицу.
constexpr auto TWO_LEVEL_HASH_TABLE_THRESHOLD = 30000;
/** Почему выбрано 30 000? Потому что при таком количестве элементов, в TwoLevelHashTable,
* скорее всего, хватит места на все ключи, с размером таблицы по-умолчанию
* (256 корзин по 256 ячеек, fill factor = 0.5)
* TODO Не конвертировать, если запрос выполняется в один поток.
*/
if (result.isConvertibleToTwoLevel() && result_size >= TWO_LEVEL_HASH_TABLE_THRESHOLD)
if (group_by_two_level_threshold && result.isConvertibleToTwoLevel() && result_size >= group_by_two_level_threshold)
result.convertToTwoLevel();
/// Проверка ограничений.
@ -1286,8 +1288,12 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va
mergeSingleLevelDataImpl<decltype(res->key_fixed_string)::element_type>(non_empty_data);
else if (res->type == AggregatedDataVariants::Type::keys128)
mergeSingleLevelDataImpl<decltype(res->keys128)::element_type>(non_empty_data);
else if (res->type == AggregatedDataVariants::Type::keys256)
mergeSingleLevelDataImpl<decltype(res->keys256)::element_type>(non_empty_data);
else if (res->type == AggregatedDataVariants::Type::hashed)
mergeSingleLevelDataImpl<decltype(res->hashed)::element_type>(non_empty_data);
else if (res->type == AggregatedDataVariants::Type::concat)
mergeSingleLevelDataImpl<decltype(res->concat)::element_type>(non_empty_data);
else if (res->type == AggregatedDataVariants::Type::key32_two_level)
mergeTwoLevelDataImpl<decltype(res->key32_two_level)::element_type>(non_empty_data, thread_pool.get());
else if (res->type == AggregatedDataVariants::Type::key64_two_level)
@ -1298,8 +1304,12 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va
mergeTwoLevelDataImpl<decltype(res->key_fixed_string_two_level)::element_type>(non_empty_data, thread_pool.get());
else if (res->type == AggregatedDataVariants::Type::keys128_two_level)
mergeTwoLevelDataImpl<decltype(res->keys128_two_level)::element_type>(non_empty_data, thread_pool.get());
else if (res->type == AggregatedDataVariants::Type::keys256_two_level)
mergeTwoLevelDataImpl<decltype(res->keys256_two_level)::element_type>(non_empty_data, thread_pool.get());
else if (res->type == AggregatedDataVariants::Type::hashed_two_level)
mergeTwoLevelDataImpl<decltype(res->hashed_two_level)::element_type>(non_empty_data, thread_pool.get());
else if (res->type == AggregatedDataVariants::Type::concat_two_level)
mergeTwoLevelDataImpl<decltype(res->concat_two_level)::element_type>(non_empty_data, thread_pool.get());
else if (res->type != AggregatedDataVariants::Type::without_key)
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
@ -1346,7 +1356,7 @@ void NO_INLINE Aggregator::mergeStreamsImpl(
bool inserted; /// Вставили новый ключ, или такой ключ уже был?
/// Получаем ключ для вставки в хэш-таблицу.
auto key = state.getKey(key_columns, keys_size, i, result.key_sizes, keys);
auto key = state.getKey(key_columns, keys_size, i, result.key_sizes, keys, *aggregates_pool);
data.emplace(key, it, inserted);
@ -1358,6 +1368,8 @@ void NO_INLINE Aggregator::mergeStreamsImpl(
aggregate_data = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(aggregate_data);
}
else
method.onExistingKey(key, keys, *aggregates_pool);
/// Мерджим состояния агрегатных функций.
for (size_t j = 0; j < aggregates_size; ++j)

View File

@ -763,14 +763,14 @@ void InterpreterSelectQuery::executeAggregation(BlockInputStreams & streams, Exp
{
stream = new ParallelAggregatingBlockInputStream(streams, key_names, aggregates, overflow_row, final,
settings.max_threads, settings.limits.max_rows_to_group_by, settings.limits.group_by_overflow_mode,
settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile);
settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, settings.group_by_two_level_threshold);
streams.resize(1);
}
else
stream = new AggregatingBlockInputStream(stream, key_names, aggregates, overflow_row, final,
settings.limits.max_rows_to_group_by, settings.limits.group_by_overflow_mode,
settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile);
settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, 0);
}

View File

@ -248,7 +248,7 @@ void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainP
for (size_t i = 0; i < rows; ++i)
{
UInt128 key = keys_fit_128_bits
? pack128(i, keys_size, key_columns, key_sizes)
? packFixed<UInt128>(i, keys_size, key_columns, key_sizes)
: hash128(i, keys_size, key_columns);
Inserter<STRICTNESS, Map>::insert(res, key, stored_block, i, pool);
@ -499,7 +499,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
for (size_t i = 0; i < rows; ++i)
{
UInt128 key = keys_fit_128_bits
? pack128(i, keys_size, key_columns, key_sizes)
? packFixed<UInt128>(i, keys_size, key_columns, key_sizes)
: hash128(i, keys_size, key_columns);
Adder<KIND, STRICTNESS, Map>::add(map, key, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get());

View File

@ -19,19 +19,7 @@ LogicalExpressionsOptimizer::OrWithExpression::OrWithExpression(ASTFunction * or
bool LogicalExpressionsOptimizer::OrWithExpression::operator<(const OrWithExpression & rhs) const
{
std::ptrdiff_t res1 = this->or_function - rhs.or_function;
if (res1 < 0)
return true;
if (res1 > 0)
return false;
int res2 = this->expression.compare(rhs.expression);
if (res2 < 0)
return true;
if (res2 > 0)
return false;
return false;
return std::tie(this->or_function, this->expression) < std::tie(rhs.or_function, rhs.expression);
}
LogicalExpressionsOptimizer::LogicalExpressionsOptimizer(ASTSelectQuery * select_query_, const Settings & settings_)
@ -172,7 +160,7 @@ bool LogicalExpressionsOptimizer::mayOptimizeDisjunctiveEqualityChain(const Disj
const auto & equality_functions = equalities.functions;
/// Исключаем слишком короткие цепочки.
if (equality_functions.size() < settings.min_or_chain_length_for_optimization)
if (equality_functions.size() < settings.optimize_min_equality_disjunction_chain_length)
return false;
/// Проверяем, что правые части всех равенств имеют один и тот же тип.

View File

@ -191,7 +191,7 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
res.insert(keys_fit_128_bits ? pack128(i, keys_size, key_columns, key_sizes) : hash128(i, keys_size, key_columns));
res.insert(keys_fit_128_bits ? packFixed<UInt128>(i, keys_size, key_columns, key_sizes) : hash128(i, keys_size, key_columns));
}
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
@ -510,7 +510,10 @@ void Set::executeOrdinary(const ConstColumnPlainPtrs & key_columns, ColumnUInt8:
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
vec_res[i] = negative ^ (set.end() != set.find(keys_fit_128_bits ? pack128(i, keys_size, key_columns, key_sizes) : hash128(i, keys_size, key_columns)));
vec_res[i] = negative ^
(set.end() != set.find(keys_fit_128_bits
? packFixed<UInt128>(i, keys_size, key_columns, key_sizes)
: hash128(i, keys_size, key_columns)));
}
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
@ -613,7 +616,10 @@ void Set::executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t
for (size_t j = prev_offset; j < offsets[i]; ++j)
{
/// Строим ключ
res |= negative ^ (set.end() != set.find(keys_fit_128_bits ? pack128(i, 1, nested_columns, key_sizes) : hash128(i, 1, nested_columns)));
res |= negative ^
(set.end() != set.find(keys_fit_128_bits
? packFixed<UInt128>(i, 1, nested_columns, key_sizes)
: hash128(i, 1, nested_columns)));
if (res)
break;
}

View File

@ -29,7 +29,7 @@ int main(int argc, char ** argv)
column_x.type = new DB::DataTypeInt16;
DB::ColumnInt16 * x = new DB::ColumnInt16;
column_x.column = x;
DB::PODArray<Int16> & vec_x = x->getData();
auto & vec_x = x->getData();
vec_x.resize(n);
for (size_t i = 0; i < n; ++i)
@ -73,7 +73,7 @@ int main(int argc, char ** argv)
DB::DataTypes empty_list_of_types;
aggregate_descriptions[0].function = factory.get("count", empty_list_of_types);
DB::Aggregator aggregator(key_column_numbers, aggregate_descriptions, false, 0, DB::OverflowMode::THROW, nullptr, 0);
DB::Aggregator aggregator(key_column_numbers, aggregate_descriptions, false, 0, DB::OverflowMode::THROW, nullptr, 0, 0);
{
Poco::Stopwatch stopwatch;

View File

@ -81,7 +81,7 @@ int main(int argc, char ** argv)
column_x.type = new DataTypeInt16;
ColumnInt16 * x = new ColumnInt16;
column_x.column = x;
PODArray<Int16> & vec_x = x->getData();
auto & vec_x = x->getData();
vec_x.resize(n);
for (size_t i = 0; i < n; ++i)

View File

@ -7,6 +7,7 @@
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Interpreters/Context.h>
namespace DB
{
@ -19,10 +20,18 @@ NamesAndTypesList ITableDeclaration::getColumnsList() const
}
ITableDeclaration::ColumnsListRange ITableDeclaration::getColumnsListIterator() const
{
const auto & columns = getColumnsListImpl();
return boost::join(
boost::iterator_range<NamesAndTypesList::const_iterator>(columns.begin(), columns.end()),
boost::iterator_range<NamesAndTypesList::const_iterator>(std::begin(materialized_columns), std::end(materialized_columns)));
}
bool ITableDeclaration::hasRealColumn(const String & column_name) const
{
const NamesAndTypesList & real_columns = getColumnsList();
for (auto & it : real_columns)
for (auto & it : getColumnsListIterator())
if (it.name == column_name)
return true;
return false;
@ -31,9 +40,8 @@ bool ITableDeclaration::hasRealColumn(const String & column_name) const
Names ITableDeclaration::getColumnNamesList() const
{
const NamesAndTypesList & real_columns = getColumnsList();
Names res;
for (auto & it : real_columns)
for (auto & it : getColumnsListIterator())
res.push_back(it.name);
return res;
}
@ -41,8 +49,7 @@ Names ITableDeclaration::getColumnNamesList() const
NameAndTypePair ITableDeclaration::getRealColumn(const String & column_name) const
{
const NamesAndTypesList & real_columns = getColumnsList();
for (auto & it : real_columns)
for (auto & it : getColumnsListIterator())
if (it.name == column_name)
return it;
throw Exception("There is no column " + column_name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
@ -79,7 +86,7 @@ NameAndTypePair ITableDeclaration::getColumn(const String & column_name) const
const DataTypePtr ITableDeclaration::getDataTypeByName(const String & column_name) const
{
for (const auto & column : getColumnsList())
for (const auto & column : getColumnsListIterator())
if (column.name == column_name)
return column.type;
@ -91,7 +98,7 @@ Block ITableDeclaration::getSampleBlock() const
{
Block res;
for (const auto & col : getColumnsList())
for (const auto & col : getColumnsListIterator())
res.insert({ col.type->createColumn(), col.type, col.name });
return res;

View File

@ -0,0 +1,123 @@
0 9
1 9
2 9
3 9
4 9
5 9
6 9
7 9
8 9
9 9
10 9
11 9
12 9
13 9
14 9
15 9
16 8
17 8
18 8
19 8
20 8
21 8
22 8
23 8
24 8
25 8
26 8
27 8
28 8
29 8
30 8
31 8
32 8
33 8
34 8
35 8
36 8
37 8
38 8
39 8
40 8
41 8
42 8
43 8
44 8
45 8
46 8
47 8
48 8
49 8
50 8
51 8
52 8
53 8
54 8
55 8
56 8
57 8
58 8
59 8
60 8
61 8
62 8
63 8
64 8
65 8
66 8
67 8
68 8
69 8
70 8
71 8
72 8
73 8
74 8
75 8
76 8
77 8
78 8
79 8
80 8
81 8
82 8
83 8
84 8
85 8
86 8
87 8
88 8
89 8
90 8
91 8
92 8
93 8
94 8
95 8
96 8
97 8
98 8
99 8
100 8
101 8
102 8
103 8
104 8
105 8
106 8
107 8
108 8
109 8
110 8
111 8
112 8
113 8
114 8
115 8
116 8
117 8
118 8
119 8
120 8
121 8
122 8

View File

@ -0,0 +1 @@
SELECT materialize('') AS k1, number % 123 AS k2, count() AS c FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k1, k2 ORDER BY k1, k2;

View File

@ -0,0 +1,10 @@
99999
99998
99997
99996
99995
99994
99993
99992
99991
99990

View File

@ -0,0 +1 @@
SELECT n, k FROM (SELECT number AS n, toFixedString(materialize(' '), 3) AS k FROM system.numbers LIMIT 100000) GROUP BY n, k ORDER BY n DESC, k LIMIT 10;

View File

@ -0,0 +1,823 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#include "libcpuid.h"
#include "asm-bits.h"
int cpuid_exists_by_eflags(void)
{
#if defined(PLATFORM_X64)
return 1; /* CPUID is always present on the x86_64 */
#elif defined(PLATFORM_X86)
# if defined(COMPILER_GCC)
int result;
__asm __volatile(
" pushfl\n"
" pop %%eax\n"
" mov %%eax, %%ecx\n"
" xor $0x200000, %%eax\n"
" push %%eax\n"
" popfl\n"
" pushfl\n"
" pop %%eax\n"
" xor %%ecx, %%eax\n"
" mov %%eax, %0\n"
" push %%ecx\n"
" popfl\n"
: "=m"(result)
: :"eax", "ecx", "memory");
return (result != 0);
# elif defined(COMPILER_MICROSOFT)
int result;
__asm {
pushfd
pop eax
mov ecx, eax
xor eax, 0x200000
push eax
popfd
pushfd
pop eax
xor eax, ecx
mov result, eax
push ecx
popfd
};
return (result != 0);
# else
return 0;
# endif /* COMPILER_MICROSOFT */
#else
return 0;
#endif /* PLATFORM_X86 */
}
/*
* with MSVC/AMD64, the exec_cpuid() and cpu_rdtsc() functions
* are implemented in separate .asm files. Otherwise, use inline assembly
*/
void exec_cpuid(uint32_t *regs)
{
#ifdef INLINE_ASM_SUPPORTED
#ifdef COMPILER_GCC
# ifdef PLATFORM_X64
__asm __volatile(
" mov %0, %%rdi\n"
" push %%rbx\n"
" push %%rcx\n"
" push %%rdx\n"
" mov (%%rdi), %%eax\n"
" mov 4(%%rdi), %%ebx\n"
" mov 8(%%rdi), %%ecx\n"
" mov 12(%%rdi), %%edx\n"
" cpuid\n"
" movl %%eax, (%%rdi)\n"
" movl %%ebx, 4(%%rdi)\n"
" movl %%ecx, 8(%%rdi)\n"
" movl %%edx, 12(%%rdi)\n"
" pop %%rdx\n"
" pop %%rcx\n"
" pop %%rbx\n"
:
:"m"(regs)
:"memory", "eax", "rdi"
);
# else
__asm __volatile(
" mov %0, %%edi\n"
" push %%ebx\n"
" push %%ecx\n"
" push %%edx\n"
" mov (%%edi), %%eax\n"
" mov 4(%%edi), %%ebx\n"
" mov 8(%%edi), %%ecx\n"
" mov 12(%%edi), %%edx\n"
" cpuid\n"
" mov %%eax, (%%edi)\n"
" mov %%ebx, 4(%%edi)\n"
" mov %%ecx, 8(%%edi)\n"
" mov %%edx, 12(%%edi)\n"
" pop %%edx\n"
" pop %%ecx\n"
" pop %%ebx\n"
:
:"m"(regs)
:"memory", "eax", "edi"
);
# endif /* COMPILER_GCC */
#else
# ifdef COMPILER_MICROSOFT
__asm {
push ebx
push ecx
push edx
push edi
mov edi, regs
mov eax, [edi]
mov ebx, [edi+4]
mov ecx, [edi+8]
mov edx, [edi+12]
cpuid
mov [edi], eax
mov [edi+4], ebx
mov [edi+8], ecx
mov [edi+12], edx
pop edi
pop edx
pop ecx
pop ebx
}
# else
# error "Unsupported compiler"
# endif /* COMPILER_MICROSOFT */
#endif
#endif /* INLINE_ASSEMBLY_SUPPORTED */
}
#ifdef INLINE_ASM_SUPPORTED
void cpu_rdtsc(uint64_t* result)
{
uint32_t low_part, hi_part;
#ifdef COMPILER_GCC
__asm __volatile (
" rdtsc\n"
" mov %%eax, %0\n"
" mov %%edx, %1\n"
:"=m"(low_part), "=m"(hi_part)::"memory", "eax", "edx"
);
#else
# ifdef COMPILER_MICROSOFT
__asm {
rdtsc
mov low_part, eax
mov hi_part, edx
};
# else
# error "Unsupported compiler"
# endif /* COMPILER_MICROSOFT */
#endif /* COMPILER_GCC */
*result = (uint64_t)low_part + (((uint64_t) hi_part) << 32);
}
#endif /* INLINE_ASM_SUPPORTED */
#ifdef INLINE_ASM_SUPPORTED
void busy_sse_loop(int cycles)
{
#ifdef COMPILER_GCC
#ifndef __APPLE__
# define XALIGN ".balign 16\n"
#else
# define XALIGN ".align 4\n"
#endif
__asm __volatile (
" xorps %%xmm0, %%xmm0\n"
" xorps %%xmm1, %%xmm1\n"
" xorps %%xmm2, %%xmm2\n"
" xorps %%xmm3, %%xmm3\n"
" xorps %%xmm4, %%xmm4\n"
" xorps %%xmm5, %%xmm5\n"
" xorps %%xmm6, %%xmm6\n"
" xorps %%xmm7, %%xmm7\n"
XALIGN
".bsLoop:\n"
// 0:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 1:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 2:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 3:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 4:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 5:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 6:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 7:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 8:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
// 9:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//10:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//11:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//12:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//13:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//14:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//15:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//16:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//17:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//18:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//19:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//20:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//21:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//22:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//23:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//24:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//25:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//26:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//27:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//28:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//29:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//30:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
//31:
" addps %%xmm1, %%xmm0\n"
" addps %%xmm2, %%xmm1\n"
" addps %%xmm3, %%xmm2\n"
" addps %%xmm4, %%xmm3\n"
" addps %%xmm5, %%xmm4\n"
" addps %%xmm6, %%xmm5\n"
" addps %%xmm7, %%xmm6\n"
" addps %%xmm0, %%xmm7\n"
" dec %%eax\n"
" jnz .bsLoop\n"
::"a"(cycles)
);
#else
# ifdef COMPILER_MICROSOFT
__asm {
mov eax, cycles
xorps xmm0, xmm0
xorps xmm1, xmm1
xorps xmm2, xmm2
xorps xmm3, xmm3
xorps xmm4, xmm4
xorps xmm5, xmm5
xorps xmm6, xmm6
xorps xmm7, xmm7
//--
align 16
bsLoop:
// 0:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 1:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 2:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 3:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 4:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 5:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 6:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 7:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 8:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 9:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 10:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 11:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 12:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 13:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 14:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 15:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 16:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 17:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 18:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 19:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 20:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 21:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 22:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 23:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 24:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 25:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 26:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 27:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 28:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 29:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 30:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
// 31:
addps xmm0, xmm1
addps xmm1, xmm2
addps xmm2, xmm3
addps xmm3, xmm4
addps xmm4, xmm5
addps xmm5, xmm6
addps xmm6, xmm7
addps xmm7, xmm0
//----------------------
dec eax
jnz bsLoop
}
# else
# error "Unsupported compiler"
# endif /* COMPILER_MICROSOFT */
#endif /* COMPILER_GCC */
}
#endif /* INLINE_ASSEMBLY_SUPPORTED */

View File

@ -0,0 +1,53 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef __ASM_BITS_H__
#define __ASM_BITS_H__
#include "libcpuid.h"
/* Determine Compiler: */
#if defined(_MSC_VER)
# define COMPILER_MICROSOFT
#elif defined(__GNUC__)
# define COMPILER_GCC
#endif
/* Determine Platform */
#if defined(__x86_64__) || defined(_M_AMD64)
# define PLATFORM_X64
#elif defined(__i386__) || defined(_M_IX86)
# define PLATFORM_X86
#endif
/* Under Windows/AMD64 with MSVC, inline assembly isn't supported */
#if (defined(COMPILER_GCC) && defined(PLATFORM_X64)) || defined(PLATFORM_X86)
# define INLINE_ASM_SUPPORTED
#endif
int cpuid_exists_by_eflags(void);
void exec_cpuid(uint32_t *regs);
void busy_sse_loop(int cycles);
#endif /* __ASM_BITS_H__ */

View File

@ -0,0 +1,2 @@
/* Version number of package */
#define VERSION "0.2.1"

View File

@ -0,0 +1,674 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#include "libcpuid.h"
#include "recog_intel.h"
#include "recog_amd.h"
#include "asm-bits.h"
#include "libcpuid_util.h"
#include "config.h"
#include <stdio.h>
#include <string.h>
#include <stdlib.h>
/* Implementation: */
static int _libcpiud_errno = ERR_OK;
int set_error(cpu_error_t err)
{
_libcpiud_errno = (int) err;
return (int) err;
}
static void raw_data_t_constructor(struct cpu_raw_data_t* raw)
{
memset(raw, 0, sizeof(struct cpu_raw_data_t));
}
static void cpu_id_t_constructor(struct cpu_id_t* id)
{
memset(id, 0, sizeof(struct cpu_id_t));
id->l1_data_cache = id->l1_instruction_cache = id->l2_cache = id->l3_cache = -1;
id->l1_assoc = id->l2_assoc = id->l3_assoc = -1;
id->l1_cacheline = id->l2_cacheline = id->l3_cacheline = -1;
id->sse_size = -1;
}
static int parse_token(const char* expected_token, const char *token,
const char *value, uint32_t array[][4], int limit, int *recognized)
{
char format[32];
int veax, vebx, vecx, vedx;
int index;
if (*recognized) return 1; /* already recognized */
if (strncmp(token, expected_token, strlen(expected_token))) return 1; /* not what we search for */
sprintf(format, "%s[%%d]", expected_token);
*recognized = 1;
if (1 == sscanf(token, format, &index) && index >=0 && index < limit) {
if (4 == sscanf(value, "%x%x%x%x", &veax, &vebx, &vecx, &vedx)) {
array[index][0] = veax;
array[index][1] = vebx;
array[index][2] = vecx;
array[index][3] = vedx;
return 1;
}
}
return 0;
}
/* get_total_cpus() system specific code: uses OS routines to determine total number of CPUs */
#ifdef __APPLE__
#include <unistd.h>
#include <mach/clock_types.h>
#include <mach/clock.h>
#include <mach/mach.h>
static int get_total_cpus(void)
{
kern_return_t kr;
host_basic_info_data_t basic_info;
host_info_t info = (host_info_t)&basic_info;
host_flavor_t flavor = HOST_BASIC_INFO;
mach_msg_type_number_t count = HOST_BASIC_INFO_COUNT;
kr = host_info(mach_host_self(), flavor, info, &count);
if (kr != KERN_SUCCESS) return 1;
return basic_info.avail_cpus;
}
#define GET_TOTAL_CPUS_DEFINED
#endif
#ifdef _WIN32
#include <windows.h>
static int get_total_cpus(void)
{
SYSTEM_INFO system_info;
GetSystemInfo(&system_info);
return system_info.dwNumberOfProcessors;
}
#define GET_TOTAL_CPUS_DEFINED
#endif
#if defined linux || defined __linux__ || defined __sun
#include <sys/sysinfo.h>
#include <unistd.h>
static int get_total_cpus(void)
{
return sysconf(_SC_NPROCESSORS_ONLN);
}
#define GET_TOTAL_CPUS_DEFINED
#endif
#if defined __FreeBSD__ || defined __OpenBSD__ || defined __NetBSD__ || defined __bsdi__ || defined __QNX__
#include <sys/sysctl.h>
static int get_total_cpus(void)
{
int mib[2] = { CTL_HW, HW_NCPU };
int ncpus;
size_t len = sizeof(ncpus);
if (sysctl(mib, 2, &ncpus, &len, (void *) 0, 0) != 0) return 1;
return ncpus;
}
#define GET_TOTAL_CPUS_DEFINED
#endif
#ifndef GET_TOTAL_CPUS_DEFINED
static int get_total_cpus(void)
{
static int warning_printed = 0;
if (!warning_printed) {
warning_printed = 1;
warnf("Your system is not supported by libcpuid -- don't know how to detect the\n");
warnf("total number of CPUs on your system. It will be reported as 1.\n");
printf("Please use cpu_id_t.logical_cpus field instead.\n");
}
return 1;
}
#endif /* GET_TOTAL_CPUS_DEFINED */
static void load_features_common(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
const struct feature_map_t matchtable_edx1[] = {
{ 0, CPU_FEATURE_FPU },
{ 1, CPU_FEATURE_VME },
{ 2, CPU_FEATURE_DE },
{ 3, CPU_FEATURE_PSE },
{ 4, CPU_FEATURE_TSC },
{ 5, CPU_FEATURE_MSR },
{ 6, CPU_FEATURE_PAE },
{ 7, CPU_FEATURE_MCE },
{ 8, CPU_FEATURE_CX8 },
{ 9, CPU_FEATURE_APIC },
{ 11, CPU_FEATURE_SEP },
{ 12, CPU_FEATURE_MTRR },
{ 13, CPU_FEATURE_PGE },
{ 14, CPU_FEATURE_MCA },
{ 15, CPU_FEATURE_CMOV },
{ 16, CPU_FEATURE_PAT },
{ 17, CPU_FEATURE_PSE36 },
{ 19, CPU_FEATURE_CLFLUSH },
{ 23, CPU_FEATURE_MMX },
{ 24, CPU_FEATURE_FXSR },
{ 25, CPU_FEATURE_SSE },
{ 26, CPU_FEATURE_SSE2 },
{ 28, CPU_FEATURE_HT },
};
const struct feature_map_t matchtable_ecx1[] = {
{ 0, CPU_FEATURE_PNI },
{ 3, CPU_FEATURE_MONITOR },
{ 9, CPU_FEATURE_SSSE3 },
{ 12, CPU_FEATURE_FMA3 },
{ 13, CPU_FEATURE_CX16 },
{ 19, CPU_FEATURE_SSE4_1 },
{ 21, CPU_FEATURE_X2APIC },
{ 23, CPU_FEATURE_POPCNT },
{ 29, CPU_FEATURE_F16C },
};
const struct feature_map_t matchtable_edx81[] = {
{ 11, CPU_FEATURE_SYSCALL },
{ 27, CPU_FEATURE_RDTSCP },
{ 29, CPU_FEATURE_LM },
};
const struct feature_map_t matchtable_ecx81[] = {
{ 0, CPU_FEATURE_LAHF_LM },
};
const struct feature_map_t matchtable_edx87[] = {
{ 8, CPU_FEATURE_CONSTANT_TSC },
};
if (raw->basic_cpuid[0][0] >= 1) {
match_features(matchtable_edx1, COUNT_OF(matchtable_edx1), raw->basic_cpuid[1][3], data);
match_features(matchtable_ecx1, COUNT_OF(matchtable_ecx1), raw->basic_cpuid[1][2], data);
}
if (raw->ext_cpuid[0][0] >= 0x80000001) {
match_features(matchtable_edx81, COUNT_OF(matchtable_edx81), raw->ext_cpuid[1][3], data);
match_features(matchtable_ecx81, COUNT_OF(matchtable_ecx81), raw->ext_cpuid[1][2], data);
}
if (raw->ext_cpuid[0][0] >= 0x80000007) {
match_features(matchtable_edx87, COUNT_OF(matchtable_edx87), raw->ext_cpuid[7][3], data);
}
if (data->flags[CPU_FEATURE_SSE]) {
/* apply guesswork to check if the SSE unit width is 128 bit */
switch (data->vendor) {
case VENDOR_AMD:
data->sse_size = (data->ext_family >= 16 && data->ext_family != 17) ? 128 : 64;
break;
case VENDOR_INTEL:
data->sse_size = (data->family == 6 && data->ext_model >= 15) ? 128 : 64;
break;
default:
break;
}
/* leave the CPU_FEATURE_128BIT_SSE_AUTH 0; the advanced per-vendor detection routines
* will set it accordingly if they detect the needed bit */
}
}
static int cpuid_basic_identify(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
int i, j, basic, xmodel, xfamily, ext;
char brandstr[64] = {0};
const struct { cpu_vendor_t vendor; char match[16]; }
matchtable[NUM_CPU_VENDORS] = {
/* source: http://www.sandpile.org/ia32/cpuid.htm */
{ VENDOR_INTEL , "GenuineIntel" },
{ VENDOR_AMD , "AuthenticAMD" },
{ VENDOR_CYRIX , "CyrixInstead" },
{ VENDOR_NEXGEN , "NexGenDriven" },
{ VENDOR_TRANSMETA , "GenuineTMx86" },
{ VENDOR_UMC , "UMC UMC UMC " },
{ VENDOR_CENTAUR , "CentaurHauls" },
{ VENDOR_RISE , "RiseRiseRise" },
{ VENDOR_SIS , "SiS SiS SiS " },
{ VENDOR_NSC , "Geode by NSC" },
};
memcpy(data->vendor_str + 0, &raw->basic_cpuid[0][1], 4);
memcpy(data->vendor_str + 4, &raw->basic_cpuid[0][3], 4);
memcpy(data->vendor_str + 8, &raw->basic_cpuid[0][2], 4);
data->vendor_str[12] = 0;
/* Determine vendor: */
data->vendor = VENDOR_UNKNOWN;
for (i = 0; i < NUM_CPU_VENDORS; i++)
if (!strcmp(data->vendor_str, matchtable[i].match)) {
data->vendor = matchtable[i].vendor;
break;
}
if (data->vendor == VENDOR_UNKNOWN)
return set_error(ERR_CPU_UNKN);
basic = raw->basic_cpuid[0][0];
if (basic >= 1) {
data->family = (raw->basic_cpuid[1][0] >> 8) & 0xf;
data->model = (raw->basic_cpuid[1][0] >> 4) & 0xf;
data->stepping = raw->basic_cpuid[1][0] & 0xf;
xmodel = (raw->basic_cpuid[1][0] >> 16) & 0xf;
xfamily = (raw->basic_cpuid[1][0] >> 20) & 0xff;
if (data->vendor == VENDOR_AMD && data->family < 0xf)
data->ext_family = data->family;
else
data->ext_family = data->family + xfamily;
data->ext_model = data->model + (xmodel << 4);
}
ext = raw->ext_cpuid[0][0] - 0x8000000;
/* obtain the brand string, if present: */
if (ext >= 4) {
for (i = 0; i < 3; i++)
for (j = 0; j < 4; j++)
memcpy(brandstr + i * 16 + j * 4,
&raw->ext_cpuid[2 + i][j], 4);
brandstr[48] = 0;
i = 0;
while (brandstr[i] == ' ') i++;
strncpy(data->brand_str, brandstr + i, sizeof(data->brand_str));
data->brand_str[48] = 0;
}
load_features_common(raw, data);
data->total_logical_cpus = get_total_cpus();
return set_error(ERR_OK);
}
static void make_list_from_string(const char* csv, struct cpu_list_t* list)
{
int i, n, l, last;
l = (int) strlen(csv);
n = 0;
for (i = 0; i < l; i++) if (csv[i] == ',') n++;
n++;
list->num_entries = n;
list->names = (char**) malloc(sizeof(char*) * n);
last = -1;
n = 0;
for (i = 0; i <= l; i++) if (i == l || csv[i] == ',') {
list->names[n] = (char*) malloc(i - last);
memcpy(list->names[n], &csv[last + 1], i - last - 1);
list->names[n][i - last - 1] = '\0';
n++;
last = i;
}
}
/* Interface: */
int cpuid_get_total_cpus(void)
{
return get_total_cpus();
}
int cpuid_present(void)
{
return cpuid_exists_by_eflags();
}
void cpu_exec_cpuid(uint32_t eax, uint32_t* regs)
{
regs[0] = eax;
regs[1] = regs[2] = regs[3] = 0;
exec_cpuid(regs);
}
void cpu_exec_cpuid_ext(uint32_t* regs)
{
exec_cpuid(regs);
}
int cpuid_get_raw_data(struct cpu_raw_data_t* data)
{
unsigned i;
if (!cpuid_present())
return set_error(ERR_NO_CPUID);
for (i = 0; i < 32; i++)
cpu_exec_cpuid(i, data->basic_cpuid[i]);
for (i = 0; i < 32; i++)
cpu_exec_cpuid(0x80000000 + i, data->ext_cpuid[i]);
for (i = 0; i < 4; i++) {
memset(data->intel_fn4[i], 0, sizeof(data->intel_fn4[i]));
data->intel_fn4[i][0] = 4;
data->intel_fn4[i][2] = i;
cpu_exec_cpuid_ext(data->intel_fn4[i]);
}
for (i = 0; i < MAX_INTELFN11_LEVEL; i++) {
memset(data->intel_fn11[i], 0, sizeof(data->intel_fn11[i]));
data->intel_fn11[i][0] = 11;
data->intel_fn11[i][2] = i;
cpu_exec_cpuid_ext(data->intel_fn11[i]);
}
return set_error(ERR_OK);
}
int cpuid_serialize_raw_data(struct cpu_raw_data_t* data, const char* filename)
{
int i;
FILE *f;
if (!strcmp(filename, ""))
f = stdout;
else
f = fopen(filename, "wt");
if (!f) return set_error(ERR_OPEN);
fprintf(f, "version=%s\n", VERSION);
for (i = 0; i < MAX_CPUID_LEVEL; i++)
fprintf(f, "basic_cpuid[%d]=%08x %08x %08x %08x\n", i,
data->basic_cpuid[i][0], data->basic_cpuid[i][1],
data->basic_cpuid[i][2], data->basic_cpuid[i][3]);
for (i = 0; i < MAX_EXT_CPUID_LEVEL; i++)
fprintf(f, "ext_cpuid[%d]=%08x %08x %08x %08x\n", i,
data->ext_cpuid[i][0], data->ext_cpuid[i][1],
data->ext_cpuid[i][2], data->ext_cpuid[i][3]);
for (i = 0; i < MAX_INTELFN4_LEVEL; i++)
fprintf(f, "intel_fn4[%d]=%08x %08x %08x %08x\n", i,
data->intel_fn4[i][0], data->intel_fn4[i][1],
data->intel_fn4[i][2], data->intel_fn4[i][3]);
for (i = 0; i < MAX_INTELFN11_LEVEL; i++)
fprintf(f, "intel_fn11[%d]=%08x %08x %08x %08x\n", i,
data->intel_fn11[i][0], data->intel_fn11[i][1],
data->intel_fn11[i][2], data->intel_fn11[i][3]);
if (strcmp(filename, ""))
fclose(f);
return set_error(ERR_OK);
}
int cpuid_deserialize_raw_data(struct cpu_raw_data_t* data, const char* filename)
{
int i, len;
char line[100];
char token[100];
char *value;
int syntax;
int cur_line = 0;
int recognized;
FILE *f;
raw_data_t_constructor(data);
if (!strcmp(filename, ""))
f = stdin;
else
f = fopen(filename, "rt");
if (!f) return set_error(ERR_OPEN);
while (fgets(line, sizeof(line), f)) {
++cur_line;
len = (int) strlen(line);
if (len < 2) continue;
if (line[len - 1] == '\n')
line[--len] = '\0';
for (i = 0; i < len && line[i] != '='; i++)
if (i >= len && i < 1 && len - i - 1 <= 0) {
fclose(f);
return set_error(ERR_BADFMT);
}
strncpy(token, line, i);
token[i] = '\0';
value = &line[i + 1];
/* try to recognize the line */
recognized = 0;
if (!strcmp(token, "version") || !strcmp(token, "build_date")) {
recognized = 1;
}
syntax = 1;
syntax = syntax && parse_token("basic_cpuid", token, value, data->basic_cpuid, 32, &recognized);
syntax = syntax && parse_token("ext_cpuid", token, value, data->ext_cpuid, 32, &recognized);
syntax = syntax && parse_token("intel_fn4", token, value, data->intel_fn4, 4, &recognized);
syntax = syntax && parse_token("intel_fn11", token, value, data->intel_fn11, 4, &recognized);
if (!syntax) {
warnf("Error: %s:%d: Syntax error\n", filename, cur_line);
fclose(f);
return set_error(ERR_BADFMT);
}
if (!recognized) {
warnf("Warning: %s:%d not understood!\n", filename, cur_line);
}
}
if (strcmp(filename, ""))
fclose(f);
return set_error(ERR_OK);
}
int cpu_identify(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
int r;
struct cpu_raw_data_t myraw;
if (!raw) {
if ((r = cpuid_get_raw_data(&myraw)) < 0)
return set_error(r);
raw = &myraw;
}
cpu_id_t_constructor(data);
if ((r = cpuid_basic_identify(raw, data)) < 0)
return set_error(r);
switch (data->vendor) {
case VENDOR_INTEL:
r = cpuid_identify_intel(raw, data);
break;
case VENDOR_AMD:
r = cpuid_identify_amd(raw, data);
break;
default:
break;
}
return set_error(r);
}
const char* cpu_feature_str(cpu_feature_t feature)
{
const struct { cpu_feature_t feature; const char* name; }
matchtable[] = {
{ CPU_FEATURE_FPU, "fpu" },
{ CPU_FEATURE_VME, "vme" },
{ CPU_FEATURE_DE, "de" },
{ CPU_FEATURE_PSE, "pse" },
{ CPU_FEATURE_TSC, "tsc" },
{ CPU_FEATURE_MSR, "msr" },
{ CPU_FEATURE_PAE, "pae" },
{ CPU_FEATURE_MCE, "mce" },
{ CPU_FEATURE_CX8, "cx8" },
{ CPU_FEATURE_APIC, "apic" },
{ CPU_FEATURE_MTRR, "mtrr" },
{ CPU_FEATURE_SEP, "sep" },
{ CPU_FEATURE_PGE, "pge" },
{ CPU_FEATURE_MCA, "mca" },
{ CPU_FEATURE_CMOV, "cmov" },
{ CPU_FEATURE_PAT, "pat" },
{ CPU_FEATURE_PSE36, "pse36" },
{ CPU_FEATURE_PN, "pn" },
{ CPU_FEATURE_CLFLUSH, "clflush" },
{ CPU_FEATURE_DTS, "dts" },
{ CPU_FEATURE_ACPI, "acpi" },
{ CPU_FEATURE_MMX, "mmx" },
{ CPU_FEATURE_FXSR, "fxsr" },
{ CPU_FEATURE_SSE, "sse" },
{ CPU_FEATURE_SSE2, "sse2" },
{ CPU_FEATURE_SS, "ss" },
{ CPU_FEATURE_HT, "ht" },
{ CPU_FEATURE_TM, "tm" },
{ CPU_FEATURE_IA64, "ia64" },
{ CPU_FEATURE_PBE, "pbe" },
{ CPU_FEATURE_PNI, "pni" },
{ CPU_FEATURE_PCLMUL, "pclmul" },
{ CPU_FEATURE_DTS64, "dts64" },
{ CPU_FEATURE_MONITOR, "monitor" },
{ CPU_FEATURE_DS_CPL, "ds_cpl" },
{ CPU_FEATURE_VMX, "vmx" },
{ CPU_FEATURE_SMX, "smx" },
{ CPU_FEATURE_EST, "est" },
{ CPU_FEATURE_TM2, "tm2" },
{ CPU_FEATURE_SSSE3, "ssse3" },
{ CPU_FEATURE_CID, "cid" },
{ CPU_FEATURE_CX16, "cx16" },
{ CPU_FEATURE_XTPR, "xtpr" },
{ CPU_FEATURE_PDCM, "pdcm" },
{ CPU_FEATURE_DCA, "dca" },
{ CPU_FEATURE_SSE4_1, "sse4_1" },
{ CPU_FEATURE_SSE4_2, "sse4_2" },
{ CPU_FEATURE_SYSCALL, "syscall" },
{ CPU_FEATURE_XD, "xd" },
{ CPU_FEATURE_X2APIC, "x2apic"},
{ CPU_FEATURE_MOVBE, "movbe" },
{ CPU_FEATURE_POPCNT, "popcnt" },
{ CPU_FEATURE_AES, "aes" },
{ CPU_FEATURE_XSAVE, "xsave" },
{ CPU_FEATURE_OSXSAVE, "osxsave" },
{ CPU_FEATURE_AVX, "avx" },
{ CPU_FEATURE_MMXEXT, "mmxext" },
{ CPU_FEATURE_3DNOW, "3dnow" },
{ CPU_FEATURE_3DNOWEXT, "3dnowext" },
{ CPU_FEATURE_NX, "nx" },
{ CPU_FEATURE_FXSR_OPT, "fxsr_opt" },
{ CPU_FEATURE_RDTSCP, "rdtscp" },
{ CPU_FEATURE_LM, "lm" },
{ CPU_FEATURE_LAHF_LM, "lahf_lm" },
{ CPU_FEATURE_CMP_LEGACY, "cmp_legacy" },
{ CPU_FEATURE_SVM, "svm" },
{ CPU_FEATURE_SSE4A, "sse4a" },
{ CPU_FEATURE_MISALIGNSSE, "misalignsse" },
{ CPU_FEATURE_ABM, "abm" },
{ CPU_FEATURE_3DNOWPREFETCH, "3dnowprefetch" },
{ CPU_FEATURE_OSVW, "osvw" },
{ CPU_FEATURE_IBS, "ibs" },
{ CPU_FEATURE_SSE5, "sse5" },
{ CPU_FEATURE_SKINIT, "skinit" },
{ CPU_FEATURE_WDT, "wdt" },
{ CPU_FEATURE_TS, "ts" },
{ CPU_FEATURE_FID, "fid" },
{ CPU_FEATURE_VID, "vid" },
{ CPU_FEATURE_TTP, "ttp" },
{ CPU_FEATURE_TM_AMD, "tm_amd" },
{ CPU_FEATURE_STC, "stc" },
{ CPU_FEATURE_100MHZSTEPS, "100mhzsteps" },
{ CPU_FEATURE_HWPSTATE, "hwpstate" },
{ CPU_FEATURE_CONSTANT_TSC, "constant_tsc" },
{ CPU_FEATURE_XOP, "xop" },
{ CPU_FEATURE_FMA3, "fma3" },
{ CPU_FEATURE_FMA4, "fma4" },
{ CPU_FEATURE_TBM, "tbm" },
{ CPU_FEATURE_F16C, "f16c" },
{ CPU_FEATURE_RDRAND, "rdrand" },
{ CPU_FEATURE_CPB, "cpb" },
{ CPU_FEATURE_APERFMPERF, "aperfmperf" },
{ CPU_FEATURE_PFI, "pfi" },
{ CPU_FEATURE_PA, "pa" },
{ CPU_FEATURE_AVX2, "avx2" },
};
unsigned i, n = COUNT_OF(matchtable);
if (n != NUM_CPU_FEATURES) {
warnf("Warning: incomplete library, feature matchtable size differs from the actual number of features.\n");
}
for (i = 0; i < n; i++)
if (matchtable[i].feature == feature)
return matchtable[i].name;
return "";
}
const char* cpuid_error(void)
{
const struct { cpu_error_t error; const char *description; }
matchtable[] = {
{ ERR_OK , "No error"},
{ ERR_NO_CPUID , "CPUID instruction is not supported"},
{ ERR_NO_RDTSC , "RDTSC instruction is not supported"},
{ ERR_NO_MEM , "Memory allocation failed"},
{ ERR_OPEN , "File open operation failed"},
{ ERR_BADFMT , "Bad file format"},
{ ERR_NOT_IMP , "Not implemented"},
{ ERR_CPU_UNKN , "Unsupported processor"},
};
unsigned i;
for (i = 0; i < COUNT_OF(matchtable); i++)
if (_libcpiud_errno == matchtable[i].error)
return matchtable[i].description;
return "Unknown error";
}
const char* cpuid_lib_version(void)
{
return VERSION;
}
libcpuid_warn_fn_t cpuid_set_warn_function(libcpuid_warn_fn_t new_fn)
{
libcpuid_warn_fn_t ret = _warn_fun;
_warn_fun = new_fn;
return ret;
}
void cpuid_set_verbosiness_level(int level)
{
_current_verboselevel = level;
}
void cpuid_get_cpu_list(cpu_vendor_t vendor, struct cpu_list_t* list)
{
switch (vendor) {
case VENDOR_INTEL:
cpuid_get_list_intel(list);
break;
case VENDOR_AMD:
cpuid_get_list_amd(list);
break;
case VENDOR_CYRIX:
make_list_from_string("Cx486,Cx5x86,6x86,6x86MX,M II,MediaGX,MediaGXi,MediaGXm", list);
break;
case VENDOR_NEXGEN:
make_list_from_string("Nx586", list);
break;
case VENDOR_TRANSMETA:
make_list_from_string("Crusoe,Efficeon", list);
break;
case VENDOR_UMC:
make_list_from_string("UMC x86 CPU", list);
break;
case VENDOR_CENTAUR:
make_list_from_string("VIA C3,VIA C7,VIA Nano", list);
break;
case VENDOR_RISE:
make_list_from_string("Rise mP6", list);
break;
case VENDOR_SIS:
make_list_from_string("SiS mP6", list);
break;
case VENDOR_NSC:
make_list_from_string("Geode GXm,Geode GXLV,Geode GX1,Geode GX2", list);
break;
default:
warnf("Unknown vendor passed to cpuid_get_cpu_list()\n");
break;
}
}
void cpuid_free_cpu_list(struct cpu_list_t* list)
{
int i;
if (list->num_entries <= 0) return;
for (i = 0; i < list->num_entries; i++)
free(list->names[i]);
free(list->names);
}

View File

@ -0,0 +1,885 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef __LIBCPUID_H__
#define __LIBCPUID_H__
/**
* @File libcpuid.h
* @Author Veselin Georgiev
* @Date Oct 2008
* @Version 0.2.1
*
* Version history:
*
* 0.1.0 (2008-10-15): initial adaptation from wxfractgui sources
* 0.1.1 (2009-07-06): Added intel_fn11 fields to cpu_raw_data_t to handle
* new processor topology enumeration required on Core i7
* 0.1.2 (2009-09-26): Added support for MSR reading through self-extracting
* kernel driver on Win32.
* 0.1.3 (2010-04-20): Added support for greater more accurate CPU clock
* measurements with cpu_clock_by_ic()
* 0.2.0 (2011-10-11): Support for AMD Bulldozer CPUs, 128-bit SSE unit size
* checking. A backwards-incompatible change, since the
* sizeof cpu_id_t is now different.
* 0.2.1 (2012-05-26): Support for Ivy Bridge, and detecting the presence of
* the RdRand instruction.
*/
/** @mainpage A simple libcpuid introduction
*
* LibCPUID provides CPU identification and access to the CPUID and RDTSC
* instructions on the x86.
* <p>
* To execute CPUID, use \ref cpu_exec_cpuid <br>
* To execute RDTSC, use \ref cpu_rdtsc <br>
* To fetch the CPUID info needed for CPU identification, use
* \ref cpuid_get_raw_data <br>
* To make sense of that data (decode, extract features), use \ref cpu_identify <br>
* </p>
*/
/** @defgroup libcpuid LibCPUID
@{ */
/* Include some integer type specifications: */
#include "libcpuid_types.h"
/* Some limits and other constants */
#include "libcpuid_constants.h"
#ifdef __cplusplus
extern "C" {
#endif
/**
* @brief CPU vendor, as guessed from the Vendor String.
*/
typedef enum {
VENDOR_INTEL = 0, /*!< Intel CPU */
VENDOR_AMD, /*!< AMD CPU */
VENDOR_CYRIX, /*!< Cyrix CPU */
VENDOR_NEXGEN, /*!< NexGen CPU */
VENDOR_TRANSMETA, /*!< Transmeta CPU */
VENDOR_UMC, /*!< x86 CPU by UMC */
VENDOR_CENTAUR, /*!< x86 CPU by IDT */
VENDOR_RISE, /*!< x86 CPU by Rise Technology */
VENDOR_SIS, /*!< x86 CPU by SiS */
VENDOR_NSC, /*!< x86 CPU by National Semiconductor */
NUM_CPU_VENDORS, /*!< Valid CPU vendor ids: 0..NUM_CPU_VENDORS - 1 */
VENDOR_UNKNOWN = -1,
} cpu_vendor_t;
#define NUM_CPU_VENDORS NUM_CPU_VENDORS
/**
* @brief Contains just the raw CPUID data.
*
* This contains only the most basic CPU data, required to do identification
* and feature recognition. Every processor should be identifiable using this
* data only.
*/
struct cpu_raw_data_t {
/** contains results of CPUID for eax = 0, 1, ...*/
uint32_t basic_cpuid[MAX_CPUID_LEVEL][4];
/** contains results of CPUID for eax = 0x80000000, 0x80000001, ...*/
uint32_t ext_cpuid[MAX_EXT_CPUID_LEVEL][4];
/** when the CPU is intel and it supports deterministic cache
information: this contains the results of CPUID for eax = 4
and ecx = 0, 1, ... */
uint32_t intel_fn4[MAX_INTELFN4_LEVEL][4];
/** when the CPU is intel and it supports leaf 0Bh (Extended Topology
enumeration leaf), this stores the result of CPUID with
eax = 11 and ecx = 0, 1, 2... */
uint32_t intel_fn11[MAX_INTELFN11_LEVEL][4];
};
/**
* @brief This contains the recognized CPU features/info
*/
struct cpu_id_t {
/** contains the CPU vendor string, e.g. "GenuineIntel" */
char vendor_str[VENDOR_STR_MAX];
/** contains the brand string, e.g. "Intel(R) Xeon(TM) CPU 2.40GHz" */
char brand_str[BRAND_STR_MAX];
/** contains the recognized CPU vendor */
cpu_vendor_t vendor;
/**
* contain CPU flags. Used to test for features. See
* the CPU_FEATURE_* macros below. @see Features
*/
uint8_t flags[CPU_FLAGS_MAX];
/** CPU family */
int32_t family;
/** CPU model */
int32_t model;
/** CPU stepping */
int32_t stepping;
/** CPU extended family */
int32_t ext_family;
/** CPU extended model */
int32_t ext_model;
/** Number of CPU cores on the current processor */
int32_t num_cores;
/**
* Number of logical processors on the current processor.
* Could be more than the number of physical cores,
* e.g. when the processor has HyperThreading.
*/
int32_t num_logical_cpus;
/**
* The total number of logical processors.
*
* This is num_logical_cpus * {total physical processors in the system}
*
* If you're writing a multithreaded program and you want to run it on
* all CPUs, this is the number of threads you need.
*/
int32_t total_logical_cpus;
/**
* L1 data cache size in KB. Could be zero, if the CPU lacks cache.
* If the size cannot be determined, it will be -1.
*/
int32_t l1_data_cache;
/**
* L1 instruction cache size in KB. Could be zero, if the CPU lacks
* cache. If the size cannot be determined, it will be -1.
* @note On some Intel CPUs, whose instruction cache is in fact
* a trace cache, the size will be expressed in K uOps.
*/
int32_t l1_instruction_cache;
/**
* L2 cache size in KB. Could be zero, if the CPU lacks L2 cache.
* If the size of the cache could not be determined, it will be -1
*/
int32_t l2_cache;
/** L3 cache size in KB. Zero on most systems */
int32_t l3_cache;
/** Cache associativity for the L1 data cache. -1 if undetermined */
int32_t l1_assoc;
/** Cache associativity for the L2 cache. -1 if undetermined */
int32_t l2_assoc;
/** Cache associativity for the L3 cache. -1 if undetermined */
int32_t l3_assoc;
/** Cache-line size for L1 data cache. -1 if undetermined */
int32_t l1_cacheline;
/** Cache-line size for L2 cache. -1 if undetermined */
int32_t l2_cacheline;
/** Cache-line size for L3 cache. -1 if undetermined */
int32_t l3_cacheline;
/**
* The brief and human-friendly CPU codename, which was recognized.<br>
* Examples:
* @code
* +--------+--------+-------+-------+-------+---------------------------------------+-----------------------+
* | Vendor | Family | Model | Step. | Cache | Brand String | cpu_id_t.cpu_codename |
* +--------+--------+-------+-------+-------+---------------------------------------+-----------------------+
* | AMD | 6 | 8 | 0 | 256 | (not available - will be ignored) | "K6-2" |
* | Intel | 15 | 2 | 5 | 512 | "Intel(R) Xeon(TM) CPU 2.40GHz" | "Xeon (Prestonia)" |
* | Intel | 6 | 15 | 11 | 4096 | "Intel(R) Core(TM)2 Duo CPU E6550..." | "Conroe (Core 2 Duo)" |
* | AMD | 15 | 35 | 2 | 1024 | "Dual Core AMD Opteron(tm) Proces..." | "Opteron (Dual Core)" |
* +--------+--------+-------+-------+-------+---------------------------------------+-----------------------+
* @endcode
*/
char cpu_codename[64];
/** SSE execution unit size (64 or 128; -1 if N/A) */
int32_t sse_size;
/**
* contain miscellaneous detection information. Used to test about specifics of
* certain detected features. See CPU_HINT_* macros below. @see Hints
*/
uint8_t detection_hints[CPU_HINTS_MAX];
};
/**
* @brief CPU feature identifiers
*
* Usage:
* @code
* ...
* struct cpu_raw_data_t raw;
* struct cpu_id_t id;
* if (cpuid_get_raw_data(&raw) == 0 && cpu_identify(&raw, &id) == 0) {
* if (id.flags[CPU_FEATURE_SSE2]) {
* // The CPU has SSE2...
* ...
* } else {
* // no SSE2
* }
* } else {
* // processor cannot be determined.
* }
* @endcode
*/
typedef enum {
CPU_FEATURE_FPU = 0, /*!< Floating point unit */
CPU_FEATURE_VME, /*!< Virtual mode extension */
CPU_FEATURE_DE, /*!< Debugging extension */
CPU_FEATURE_PSE, /*!< Page size extension */
CPU_FEATURE_TSC, /*!< Time-stamp counter */
CPU_FEATURE_MSR, /*!< Model-specific regsisters, RDMSR/WRMSR supported */
CPU_FEATURE_PAE, /*!< Physical address extension */
CPU_FEATURE_MCE, /*!< Machine check exception */
CPU_FEATURE_CX8, /*!< CMPXCHG8B instruction supported */
CPU_FEATURE_APIC, /*!< APIC support */
CPU_FEATURE_MTRR, /*!< Memory type range registers */
CPU_FEATURE_SEP, /*!< SYSENTER / SYSEXIT instructions supported */
CPU_FEATURE_PGE, /*!< Page global enable */
CPU_FEATURE_MCA, /*!< Machine check architecture */
CPU_FEATURE_CMOV, /*!< CMOVxx instructions supported */
CPU_FEATURE_PAT, /*!< Page attribute table */
CPU_FEATURE_PSE36, /*!< 36-bit page address extension */
CPU_FEATURE_PN, /*!< Processor serial # implemented (Intel P3 only) */
CPU_FEATURE_CLFLUSH, /*!< CLFLUSH instruction supported */
CPU_FEATURE_DTS, /*!< Debug store supported */
CPU_FEATURE_ACPI, /*!< ACPI support (power states) */
CPU_FEATURE_MMX, /*!< MMX instruction set supported */
CPU_FEATURE_FXSR, /*!< FXSAVE / FXRSTOR supported */
CPU_FEATURE_SSE, /*!< Streaming-SIMD Extensions (SSE) supported */
CPU_FEATURE_SSE2, /*!< SSE2 instructions supported */
CPU_FEATURE_SS, /*!< Self-snoop */
CPU_FEATURE_HT, /*!< Hyper-threading supported (but might be disabled) */
CPU_FEATURE_TM, /*!< Thermal monitor */
CPU_FEATURE_IA64, /*!< IA64 supported (Itanium only) */
CPU_FEATURE_PBE, /*!< Pending-break enable */
CPU_FEATURE_PNI, /*!< PNI (SSE3) instructions supported */
CPU_FEATURE_PCLMUL, /*!< PCLMULQDQ instruction supported */
CPU_FEATURE_DTS64, /*!< 64-bit Debug store supported */
CPU_FEATURE_MONITOR, /*!< MONITOR / MWAIT supported */
CPU_FEATURE_DS_CPL, /*!< CPL Qualified Debug Store */
CPU_FEATURE_VMX, /*!< Virtualization technology supported */
CPU_FEATURE_SMX, /*!< Safer mode exceptions */
CPU_FEATURE_EST, /*!< Enhanced SpeedStep */
CPU_FEATURE_TM2, /*!< Thermal monitor 2 */
CPU_FEATURE_SSSE3, /*!< SSSE3 instructionss supported (this is different from SSE3!) */
CPU_FEATURE_CID, /*!< Context ID supported */
CPU_FEATURE_CX16, /*!< CMPXCHG16B instruction supported */
CPU_FEATURE_XTPR, /*!< Send Task Priority Messages disable */
CPU_FEATURE_PDCM, /*!< Performance capabilities MSR supported */
CPU_FEATURE_DCA, /*!< Direct cache access supported */
CPU_FEATURE_SSE4_1, /*!< SSE 4.1 instructions supported */
CPU_FEATURE_SSE4_2, /*!< SSE 4.2 instructions supported */
CPU_FEATURE_SYSCALL, /*!< SYSCALL / SYSRET instructions supported */
CPU_FEATURE_XD, /*!< Execute disable bit supported */
CPU_FEATURE_MOVBE, /*!< MOVBE instruction supported */
CPU_FEATURE_POPCNT, /*!< POPCNT instruction supported */
CPU_FEATURE_AES, /*!< AES* instructions supported */
CPU_FEATURE_XSAVE, /*!< XSAVE/XRSTOR/etc instructions supported */
CPU_FEATURE_OSXSAVE, /*!< non-privileged copy of OSXSAVE supported */
CPU_FEATURE_AVX, /*!< Advanced vector extensions supported */
CPU_FEATURE_MMXEXT, /*!< AMD MMX-extended instructions supported */
CPU_FEATURE_3DNOW, /*!< AMD 3DNow! instructions supported */
CPU_FEATURE_3DNOWEXT, /*!< AMD 3DNow! extended instructions supported */
CPU_FEATURE_NX, /*!< No-execute bit supported */
CPU_FEATURE_FXSR_OPT, /*!< FFXSR: FXSAVE and FXRSTOR optimizations */
CPU_FEATURE_RDTSCP, /*!< RDTSCP instruction supported (AMD-only) */
CPU_FEATURE_LM, /*!< Long mode (x86_64/EM64T) supported */
CPU_FEATURE_LAHF_LM, /*!< LAHF/SAHF supported in 64-bit mode */
CPU_FEATURE_CMP_LEGACY, /*!< core multi-processing legacy mode */
CPU_FEATURE_SVM, /*!< AMD Secure virtual machine */
CPU_FEATURE_ABM, /*!< LZCNT instruction support */
CPU_FEATURE_MISALIGNSSE,/*!< Misaligned SSE supported */
CPU_FEATURE_SSE4A, /*!< SSE 4a from AMD */
CPU_FEATURE_3DNOWPREFETCH, /*!< PREFETCH/PREFETCHW support */
CPU_FEATURE_OSVW, /*!< OS Visible Workaround (AMD) */
CPU_FEATURE_IBS, /*!< Instruction-based sampling */
CPU_FEATURE_SSE5, /*!< SSE 5 instructions supported (deprecated, will never be 1) */
CPU_FEATURE_SKINIT, /*!< SKINIT / STGI supported */
CPU_FEATURE_WDT, /*!< Watchdog timer support */
CPU_FEATURE_TS, /*!< Temperature sensor */
CPU_FEATURE_FID, /*!< Frequency ID control */
CPU_FEATURE_VID, /*!< Voltage ID control */
CPU_FEATURE_TTP, /*!< THERMTRIP */
CPU_FEATURE_TM_AMD, /*!< AMD-specified hardware thermal control */
CPU_FEATURE_STC, /*!< Software thermal control */
CPU_FEATURE_100MHZSTEPS,/*!< 100 MHz multiplier control */
CPU_FEATURE_HWPSTATE, /*!< Hardware P-state control */
CPU_FEATURE_CONSTANT_TSC, /*!< TSC ticks at constant rate */
CPU_FEATURE_XOP, /*!< The XOP instruction set (same as the old CPU_FEATURE_SSE5) */
CPU_FEATURE_FMA3, /*!< The FMA3 instruction set */
CPU_FEATURE_FMA4, /*!< The FMA4 instruction set */
CPU_FEATURE_TBM, /*!< Trailing bit manipulation instruction support */
CPU_FEATURE_F16C, /*!< 16-bit FP convert instruction support */
CPU_FEATURE_RDRAND, /*!< RdRand instruction */
CPU_FEATURE_X2APIC, /*!< x2APIC, APIC_BASE.EXTD, MSRs 0000_0800h...0000_0BFFh 64-bit ICR (+030h but not +031h), no DFR (+00Eh), SELF_IPI (+040h) also see standard level 0000_000Bh */
CPU_FEATURE_CPB, /*!< Core performance boost */
CPU_FEATURE_APERFMPERF, /*!< MPERF/APERF MSRs support */
CPU_FEATURE_PFI, /*!< Processor Feedback Interface support */
CPU_FEATURE_PA, /*!< Processor accumulator */
CPU_FEATURE_AVX2, /*!< AVX2 instructions */
/* termination: */
NUM_CPU_FEATURES,
} cpu_feature_t;
/**
* @brief CPU detection hints identifiers
*
* Usage: similar to the flags usage
*/
typedef enum {
CPU_HINT_SSE_SIZE_AUTH = 0, /*!< SSE unit size is authoritative (not only a Family/Model guesswork, but based on an actual CPUID bit) */
/* termination */
NUM_CPU_HINTS,
} cpu_hint_t;
/**
* @brief Describes common library error codes
*/
typedef enum {
ERR_OK = 0, /*!< "No error" */
ERR_NO_CPUID = -1, /*!< "CPUID instruction is not supported" */
ERR_NO_RDTSC = -2, /*!< "RDTSC instruction is not supported" */
ERR_NO_MEM = -3, /*!< "Memory allocation failed" */
ERR_OPEN = -4, /*!< "File open operation failed" */
ERR_BADFMT = -5, /*!< "Bad file format" */
ERR_NOT_IMP = -6, /*!< "Not implemented" */
ERR_CPU_UNKN = -7, /*!< "Unsupported processor" */
ERR_NO_RDMSR = -8, /*!< "RDMSR instruction is not supported" */
ERR_NO_DRIVER= -9, /*!< "RDMSR driver error (generic)" */
ERR_NO_PERMS = -10, /*!< "No permissions to install RDMSR driver" */
ERR_EXTRACT = -11, /*!< "Cannot extract RDMSR driver (read only media?)" */
ERR_HANDLE = -12, /*!< "Bad handle" */
ERR_INVMSR = -13, /*!< "Invalid MSR" */
} cpu_error_t;
/**
* @brief Internal structure, used in cpu_tsc_mark, cpu_tsc_unmark and
* cpu_clock_by_mark
*/
struct cpu_mark_t {
uint64_t tsc; /*!< Time-stamp from RDTSC */
uint64_t sys_clock; /*!< In microsecond resolution */
};
/**
* @brief Returns the total number of CPUs even if CPUID is not present
* @retval Number of CPUs available
*/
int cpuid_get_total_cpus(void);
/**
* @brief Checks if the CPUID instruction is supported
* @retval 1 if CPUID is present
* @retval 0 the CPU doesn't have CPUID.
*/
int cpuid_present(void);
/**
* @brief Executes the CPUID instruction
* @param eax - the value of the EAX register when executing CPUID
* @param regs - the results will be stored here. regs[0] = EAX, regs[1] = EBX, ...
* @note CPUID will be executed with EAX set to the given value and EBX, ECX,
* EDX set to zero.
*/
void cpu_exec_cpuid(uint32_t eax, uint32_t* regs);
/**
* @brief Executes the CPUID instruction with the given input registers
* @note This is just a bit more generic version of cpu_exec_cpuid - it allows
* you to control all the registers.
* @param regs - Input/output. Prior to executing CPUID, EAX, EBX, ECX and
* EDX will be set to regs[0], regs[1], regs[2] and regs[3].
* After CPUID, this array will contain the results.
*/
void cpu_exec_cpuid_ext(uint32_t* regs);
/**
* @brief Obtains the raw CPUID data from the current CPU
* @param data - a pointer to cpu_raw_data_t structure
* @returns zero if successful, and some negative number on error.
* The error message can be obtained by calling \ref cpuid_error.
* @see cpu_error_t
*/
int cpuid_get_raw_data(struct cpu_raw_data_t* data);
/**
* @brief Writes the raw CPUID data to a text file
* @param data - a pointer to cpu_raw_data_t structure
* @param filename - the path of the file, where the serialized data should be
* written. If empty, stdout will be used.
* @note This is intended primarily for debugging. On some processor, which is
* not currently supported or not completely recognized by cpu_identify,
* one can still successfully get the raw data and write it to a file.
* libcpuid developers can later import this file and debug the detection
* code as if running on the actual hardware.
* The file is simple text format of "something=value" pairs. Version info
* is also written, but the format is not intended to be neither backward-
* nor forward compatible.
* @returns zero if successful, and some negative number on error.
* The error message can be obtained by calling \ref cpuid_error.
* @see cpu_error_t
*/
int cpuid_serialize_raw_data(struct cpu_raw_data_t* data, const char* filename);
/**
* @brief Reads raw CPUID data from file
* @param data - a pointer to cpu_raw_data_t structure. The deserialized data will
* be written here.
* @param filename - the path of the file, containing the serialized raw data.
* If empty, stdin will be used.
* @note This function may fail, if the file is created by different version of
* the library. Also, see the notes on cpuid_serialize_raw_data.
* @returns zero if successful, and some negative number on error.
* The error message can be obtained by calling \ref cpuid_error.
* @see cpu_error_t
*/
int cpuid_deserialize_raw_data(struct cpu_raw_data_t* data, const char* filename);
/**
* @brief Identifies the CPU
* @param raw - Input - a pointer to the raw CPUID data, which is obtained
* either by cpuid_get_raw_data or cpuid_deserialize_raw_data.
* Can also be NULL, in which case the functions calls
* cpuid_get_raw_data itself.
* @param data - Output - the decoded CPU features/info is written here.
* @note The function will not fail, even if some of the information
* cannot be obtained. Even when the CPU is new and thus unknown to
* libcpuid, some generic info, such as "AMD K9 family CPU" will be
* written to data.cpu_codename, and most other things, such as the
* CPU flags, cache sizes, etc. should be detected correctly anyway.
* However, the function CAN fail, if the CPU is completely alien to
* libcpuid.
* @note While cpu_identify() and cpuid_get_raw_data() are fast for most
* purposes, running them several thousand times per second can hamper
* performance significantly. Specifically, avoid writing "cpu feature
* checker" wrapping function, which calls cpu_identify and returns the
* value of some flag, if that function is going to be called frequently.
* @returns zero if successful, and some negative number on error.
* The error message can be obtained by calling \ref cpuid_error.
* @see cpu_error_t
*/
int cpu_identify(struct cpu_raw_data_t* raw, struct cpu_id_t* data);
/**
* @brief Returns the short textual representation of a CPU flag
* @param feature - the feature, whose textual representation is wanted.
* @returns a constant string like "fpu", "tsc", "sse2", etc.
* @note the names of the returned flags are compatible with those from
* /proc/cpuinfo in Linux, with the exception of `tm_amd'
*/
const char* cpu_feature_str(cpu_feature_t feature);
/**
* @brief Returns textual description of the last error
*
* libcpuid stores an `errno'-style error status, whose description
* can be obtained with this function.
* @note This function is not thread-safe
* @see cpu_error_t
*/
const char* cpuid_error(void);
/**
* @brief Executes RDTSC
*
* The RDTSC (ReaD Time Stamp Counter) instruction gives access to an
* internal 64-bit counter, which usually increments at each clock cycle.
* This can be used for various timing routines, and as a very precise
* clock source. It is set to zero on system startup. Beware that may not
* increment at the same frequency as the CPU. Consecutive calls of RDTSC
* are, however, guaranteed to return monotonically-increasing values.
*
* @param result - a pointer to a 64-bit unsigned integer, where the TSC value
* will be stored
*
* @note If 100% compatibility is a concern, you must first check if the
* RDTSC instruction is present (if it is not, your program will crash
* with "invalid opcode" exception). Only some very old processors (i486,
* early AMD K5 and some Cyrix CPUs) lack that instruction - they should
* have become exceedingly rare these days. To verify RDTSC presence,
* run cpu_identify() and check flags[CPU_FEATURE_TSC].
*
* @note The monotonically increasing nature of the TSC may be violated
* on SMP systems, if their TSC clocks run at different rate. If the OS
* doesn't account for that, the TSC drift may become arbitrary large.
*/
void cpu_rdtsc(uint64_t* result);
/**
* @brief Store TSC and timing info
*
* This function stores the current TSC value and current
* time info from a precise OS-specific clock source in the cpu_mark_t
* structure. The sys_clock field contains time with microsecond resolution.
* The values can later be used to measure time intervals, number of clocks,
* FPU frequency, etc.
* @see cpu_rdtsc
*
* @param mark [out] - a pointer to a cpu_mark_t structure
*/
void cpu_tsc_mark(struct cpu_mark_t* mark);
/**
* @brief Calculate TSC and timing difference
*
* @param mark - input/output: a pointer to a cpu_mark_t sturcture, which has
* already been initialized by cpu_tsc_mark. The difference in
* TSC and time will be written here.
*
* This function calculates the TSC and time difference, by obtaining the
* current TSC and timing values and subtracting the contents of the `mark'
* structure from them. Results are written in the same structure.
*
* Example:
* @code
* ...
* struct cpu_mark_t mark;
* cpu_tsc_mark(&mark);
* foo();
* cpu_tsc_unmark(&mark);
* printf("Foo finished. Executed in %llu cycles and %llu usecs\n",
* mark.tsc, mark.sys_clock);
* ...
* @endcode
*/
void cpu_tsc_unmark(struct cpu_mark_t* mark);
/**
* @brief Calculates the CPU clock
*
* @param mark - pointer to a cpu_mark_t structure, which has been initialized
* with cpu_tsc_mark and later `stopped' with cpu_tsc_unmark.
*
* @note For reliable results, the marked time interval should be at least about
* 10 ms.
*
* @returns the CPU clock frequency, in MHz. Due to measurement error, it will
* differ from the true value in a few least-significant bits. Accuracy depends
* on the timing interval - the more, the better. If the timing interval is
* insufficient, the result is -1. Also, see the comment on cpu_clock_measure
* for additional issues and pitfalls in using RDTSC for CPU frequency
* measurements.
*/
int cpu_clock_by_mark(struct cpu_mark_t* mark);
/**
* @brief Returns the CPU clock, as reported by the OS
*
* This function uses OS-specific functions to obtain the CPU clock. It may
* differ from the true clock for several reasons:<br><br>
*
* i) The CPU might be in some power saving state, while the OS reports its
* full-power frequency, or vice-versa.<br>
* ii) In some cases you can raise or lower the CPU frequency with overclocking
* utilities and the OS will not notice.
*
* @returns the CPU clock frequency in MHz. If the OS is not (yet) supported
* or lacks the necessary reporting machinery, the return value is -1
*/
int cpu_clock_by_os(void);
/**
* @brief Measure the CPU clock frequency
*
* @param millis - How much time to waste in the busy-wait cycle. In millisecs.
* Useful values 10 - 1000
* @param quad_check - Do a more thorough measurement if nonzero
* (see the explanation).
*
* The function performs a busy-wait cycle for the given time and calculates
* the CPU frequency by the difference of the TSC values. The accuracy of the
* calculation depends on the length of the busy-wait cycle: more is better,
* but 100ms should be enough for most purposes.
*
* While this will calculate the CPU frequency correctly in most cases, there are
* several reasons why it might be incorrect:<br>
*
* i) RDTSC doesn't guarantee it will run at the same clock as the CPU.
* Apparently there aren't CPUs at the moment, but still, there's no
* guarantee.<br>
* ii) The CPU might be in a low-frequency power saving mode, and the CPU
* might be switched to higher frequency at any time. If this happens
* during the measurement, the result can be anywhere between the
* low and high frequencies. Also, if you're interested in the
* high frequency value only, this function might return the low one
* instead.<br>
* iii) On SMP systems exhibiting TSC drift (see \ref cpu_rdtsc)
*
* the quad_check option will run four consecutive measurements and
* then return the average of the two most-consistent results. The total
* runtime of the function will still be `millis' - consider using
* a bit more time for the timing interval.
*
* Finally, for benchmarking / CPU intensive applications, the best strategy is
* to use the cpu_tsc_mark() / cpu_tsc_unmark() / cpu_clock_by_mark() method.
* Begin by mark()-ing about one second after application startup (allowing the
* power-saving manager to kick in and rise the frequency during that time),
* then unmark() just before application finishing. The result will most
* acurately represent at what frequency your app was running.
*
* @returns the CPU clock frequency in MHz (within some measurement error
* margin). If RDTSC is not supported, the result is -1.
*/
int cpu_clock_measure(int millis, int quad_check);
/**
* @brief Measure the CPU clock frequency using instruction-counting
*
* @param millis - how much time to allocate for each run, in milliseconds
* @param runs - how many runs to perform
*
* The function performs a busy-wait cycle using a known number of "heavy" (SSE)
* instructions. These instructions run at (more or less guaranteed) 1 IPC rate,
* so by running a busy loop for a fixed amount of time, and measuring the
* amount of instructions done, the CPU clock is accurately measured.
*
* Of course, this function is still affected by the power-saving schemes, so
* the warnings as of cpu_clock_measure() still apply. However, this function is
* immune to problems with detection, related to the Intel Nehalem's "Turbo"
* mode, where the internal clock is raised, but the RDTSC rate is unaffected.
*
* The function will run for about (millis * runs) milliseconds.
* You can make only a single busy-wait run (runs == 1); however, this can
* be affected by task scheduling (which will break the counting), so allowing
* more than one run is recommended. As run length is not imperative for
* accurate readings (e.g., 50ms is sufficient), you can afford a lot of short
* runs, e.g. 10 runs of 50ms or 20 runs of 25ms.
*
* Recommended values - millis = 50, runs = 4. For more robustness,
* increase the number of runs.
*
* NOTE: on Bulldozer and later CPUs, the busy-wait cycle runs at 1.4 IPC, thus
* the results are skewed. This is corrected internally by dividing the resulting
* value by 1.4.
* However, this only occurs if the thread is executed on a single CMT
* module - if there are other threads competing for resources, the results are
* unpredictable. Make sure you run cpu_clock_by_ic() on a CPU that is free from
* competing threads, or if there are such threads, they shouldn't exceed the
* number of modules. On a Bulldozer X8, that means 4 threads.
*
* @returns the CPU clock frequency in MHz (within some measurement error
* margin). If SSE is not supported, the result is -1. If the input parameters
* are incorrect, or some other internal fault is detected, the result is -2.
*/
int cpu_clock_by_ic(int millis, int runs);
/**
* @brief Get the CPU clock frequency (all-in-one method)
*
* This is an all-in-one method for getting the CPU clock frequency.
* It tries to use the OS for that. If the OS doesn't have this info, it
* uses cpu_clock_measure with 200ms time interval and quadruple checking.
*
* @returns the CPU clock frequency in MHz. If every possible method fails,
* the result is -1.
*/
int cpu_clock(void);
/**
* @brief Returns the libcpuid version
*
* @returns the string representation of the libcpuid version, like "0.1.1"
*/
const char* cpuid_lib_version(void);
typedef void (*libcpuid_warn_fn_t) (const char *msg);
/**
* @brief Sets the warning print function
*
* In some cases, the internal libcpuid machinery would like to emit useful
* debug warnings. By default, these warnings are written to stderr. However,
* you can set a custom function that will receive those warnings.
*
* @param warn_fun - the warning function you want to set. If NULL, warnings
* are disabled. The function takes const char* argument.
*
* @returns the current warning function. You can use the return value to
* keep the previous warning function and restore it at your discretion.
*/
libcpuid_warn_fn_t cpuid_set_warn_function(libcpuid_warn_fn_t warn_fun);
/**
* @brief Sets the verbosiness level
*
* When the verbosiness level is above zero, some functions might print
* diagnostic information about what are they doing. The higher the level is,
* the more detail is printed. Level zero is guaranteed to omit all such
* output. The output is written using the same machinery as the warnings,
* @see cpuid_set_warn_function()
*
* @param level the desired verbosiness level. Useful values 0..2 inclusive
*/
void cpuid_set_verbosiness_level(int level);
/**
* @brief a structure that holds a list of processor names
*/
struct cpu_list_t {
/** Number of entries in the list */
int num_entries;
/** Pointers to names. There will be num_entries of them */
char **names;
};
/**
* @brief Gets a list of all known CPU names from a specific vendor.
*
* This function compiles a list of all known CPU (code)names
* (i.e. the possible values of cpu_id_t::cpu_codename) for the given vendor.
*
* There are about 100 entries for Intel and AMD, and a few for the other
* vendors. The list is written out in approximate chronological introduction
* order of the parts.
*
* @param vendor the vendor to be queried
* @param list [out] the resulting list will be written here.
* NOTE: As the memory is dynamically allocated, be sure to call
* cpuid_free_cpu_list() after you're done with the data
* @see cpu_list_t
*/
void cpuid_get_cpu_list(cpu_vendor_t vendor, struct cpu_list_t* list);
/**
* @brief Frees a CPU list
*
* This function deletes all the memory associated with a CPU list, as obtained
* by cpuid_get_cpu_list()
*
* @param list - the list to be free()'d.
*/
void cpuid_free_cpu_list(struct cpu_list_t* list);
/**
* @brief Starts/opens a driver, needed to read MSRs (Model Specific Registers)
*
* On systems that support it, this function will create a temporary
* system driver, that has privileges to execute the RDMSR instruction.
* After the driver is created, you can read MSRs by calling \ref cpu_rdmsr
*
* @returns a handle to the driver on success, and NULL on error.
* The error message can be obtained by calling \ref cpuid_error.
* @see cpu_error_t
*/
struct msr_driver_t;
struct msr_driver_t* cpu_msr_driver_open(void);
/**
* @brief Reads a Model-Specific Register (MSR)
*
* If the CPU has MSRs (as indicated by the CPU_FEATURE_MSR flag), you can
* read a MSR with the given index by calling this function.
*
* There are several prerequisites you must do before reading MSRs:
* 1) You must ensure the CPU has RDMSR. Check the CPU_FEATURE_MSR flag
* in cpu_id_t::flags
* 2) You must ensure that the CPU implements the specific MSR you intend to
* read.
* 3) You must open a MSR-reader driver. RDMSR is a privileged instruction and
* needs ring-0 access in order to work. This temporary driver is created
* by calling \ref cpu_msr_driver_open
*
* @param handle - a handle to the MSR reader driver, as created by
* cpu_msr_driver_open
* @param msr_index - the numeric ID of the MSR you want to read
* @param result - a pointer to a 64-bit integer, where the MSR value is stored
*
* @returns zero if successful, and some negative number on error.
* The error message can be obtained by calling \ref cpuid_error.
* @see cpu_error_t
*/
int cpu_rdmsr(struct msr_driver_t* handle, int msr_index, uint64_t* result);
typedef enum {
INFO_MPERF, /*!< Maximum performance frequency clock. This
is a counter, which increments as a
proportion of the actual processor speed */
INFO_APERF, /*!< Actual performance frequency clock. This
accumulates the core clock counts when the
core is active. */
INFO_CUR_MULTIPLIER, /*!< Current CPU:FSB ratio, multiplied by 100.
e.g., a CPU:FSB value of 18.5 reads as
1850. */
INFO_MAX_MULTIPLIER, /*!< Maxumum CPU:FSB ratio for this CPU,
multiplied by 100 */
INFO_TEMPERATURE, /*!< The current core temperature in Celsius */
INFO_THROTTLING, /*!< 1 if the current logical processor is
throttling. 0 if it is running normally. */
} cpu_msrinfo_request_t;
/**
* @brief Reads extended CPU information from Model-Specific Registers.
* @param handle - a handle to an open MSR driver, @see cpu_msr_driver_open
* @param which - which info field should be returned. A list of
* available information entities is listed in the
* cpu_msrinfo_request_t enum.
* @retval - if the requested information is available for the current
* processor model, the respective value is returned.
* if no information is available, or the CPU doesn't support
* the query, the special value CPU_INVALID_VALUE is returned
*/
int cpu_msrinfo(struct msr_driver_t* handle, cpu_msrinfo_request_t which);
#define CPU_INVALID_VALUE 0x3fffffff
/**
* @brief Closes an open MSR driver
*
* This function unloads the MSR driver opened by cpu_msr_driver_open and
* frees any resources associated with it.
*
* @param handle - a handle to the MSR reader driver, as created by
* cpu_msr_driver_open
*
* @returns zero if successful, and some negative number on error.
* The error message can be obtained by calling \ref cpuid_error.
* @see cpu_error_t
*/
int cpu_msr_driver_close(struct msr_driver_t* handle);
#ifdef __cplusplus
}; /* extern "C" */
#endif
/** @} */
#endif /* __LIBCPUID_H__ */

View File

@ -0,0 +1,44 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
/**
* @File libcpuid_constants.h
* @Author Veselin Georgiev
* @Brief Some limits and constants for libcpuid
*/
#ifndef __LIBCPUID_CONSTANTS_H__
#define __LIBCPUID_CONSTANTS_H__
#define VENDOR_STR_MAX 16
#define BRAND_STR_MAX 64
#define CPU_FLAGS_MAX 128
#define MAX_CPUID_LEVEL 32
#define MAX_EXT_CPUID_LEVEL 32
#define MAX_INTELFN4_LEVEL 4
#define MAX_INTELFN11_LEVEL 4
#define CPU_HINTS_MAX 16
#endif /* __LIBCPUID_CONSTANTS_H__ */

View File

@ -0,0 +1,37 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
/**
* @File libcpuid_types.h
* @Author Veselin Georgiev
* @Brief Type specifications for libcpuid.
*/
#ifndef __LIBCPUID_TYPES_H__
#define __LIBCPUID_TYPES_H__
# include <stdint.h>
#endif /* __LIBCPUID_TYPES_H__ */

View File

@ -0,0 +1,182 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#include <stdio.h>
#include <stdlib.h>
#include <stdarg.h>
#include <string.h>
#include <ctype.h>
#include "libcpuid.h"
#include "libcpuid_util.h"
int _current_verboselevel;
void match_features(const struct feature_map_t* matchtable, int count, uint32_t reg, struct cpu_id_t* data)
{
int i;
for (i = 0; i < count; i++)
if (reg & (1 << matchtable[i].bit))
data->flags[matchtable[i].feature] = 1;
}
static void default_warn(const char *msg)
{
fprintf(stderr, "%s", msg);
}
libcpuid_warn_fn_t _warn_fun = default_warn;
#if defined(_MSC_VER)
# define vsnprintf _vsnprintf
#endif
void warnf(const char* format, ...)
{
char buff[1024];
va_list va;
if (!_warn_fun) return;
va_start(va, format);
vsnprintf(buff, sizeof(buff), format, va);
va_end(va);
_warn_fun(buff);
}
void debugf(int verboselevel, const char* format, ...)
{
char buff[1024];
va_list va;
if (verboselevel > _current_verboselevel) return;
va_start(va, format);
vsnprintf(buff, sizeof(buff), format, va);
va_end(va);
_warn_fun(buff);
}
static int score(const struct match_entry_t* entry, const struct cpu_id_t* data,
int brand_code, int model_code)
{
int res = 0;
if (entry->family == data->family ) res++;
if (entry->model == data->model ) res++;
if (entry->stepping == data->stepping ) res++;
if (entry->ext_family == data->ext_family) res++;
if (entry->ext_model == data->ext_model ) res++;
if (entry->ncores == data->num_cores ) res++;
if (entry->l2cache == data->l2_cache ) res++;
if (entry->l3cache == data->l3_cache ) res++;
if (entry->brand_code == brand_code ) res++;
if (entry->model_code == model_code ) res++;
return res;
}
void match_cpu_codename(const struct match_entry_t* matchtable, int count,
struct cpu_id_t* data, int brand_code, int model_code)
{
int bestscore = -1;
int bestindex = 0;
int i, t;
debugf(3, "Matching cpu f:%d, m:%d, s:%d, xf:%d, xm:%d, ncore:%d, l2:%d, bcode:%d, code:%d\n",
data->family, data->model, data->stepping, data->ext_family,
data->ext_model, data->num_cores, data->l2_cache, brand_code, model_code);
for (i = 0; i < count; i++) {
t = score(&matchtable[i], data, brand_code, model_code);
debugf(3, "Entry %d, `%s', score %d\n", i, matchtable[i].name, t);
if (t > bestscore) {
debugf(2, "Entry `%s' selected - best score so far (%d)\n", matchtable[i].name, t);
bestscore = t;
bestindex = i;
}
}
strcpy(data->cpu_codename, matchtable[bestindex].name);
}
void generic_get_cpu_list(const struct match_entry_t* matchtable, int count,
struct cpu_list_t* list)
{
int i, j, n, good;
n = 0;
list->names = (char**) malloc(sizeof(char*) * count);
for (i = 0; i < count; i++) {
if (strstr(matchtable[i].name, "Unknown")) continue;
good = 1;
for (j = n - 1; j >= 0; j--)
if (!strcmp(list->names[j], matchtable[i].name)) {
good = 0;
break;
}
if (!good) continue;
list->names[n++] = strdup(matchtable[i].name);
}
list->num_entries = n;
}
static int xmatch_entry(char c, const char* p)
{
int i, j;
if (c == 0) return -1;
if (c == p[0]) return 1;
if (p[0] == '.') return 1;
if (p[0] == '#' && isdigit(c)) return 1;
if (p[0] == '[') {
j = 1;
while (p[j] && p[j] != ']') j++;
if (!p[j]) return -1;
for (i = 1; i < j; i++)
if (p[i] == c) return j + 1;
}
return -1;
}
int match_pattern(const char* s, const char* p)
{
int i, j, dj, k, n, m;
n = (int) strlen(s);
m = (int) strlen(p);
for (i = 0; i < n; i++) {
if (xmatch_entry(s[i], p) != -1) {
j = 0;
k = 0;
while (j < m && ((dj = xmatch_entry(s[i + k], p + j)) != -1)) {
k++;
j += dj;
}
if (j == m) return i + 1;
}
}
return 0;
}
struct cpu_id_t* get_cached_cpuid(void)
{
static int initialized = 0;
static struct cpu_id_t id;
if (initialized) return &id;
if (cpu_identify(NULL, &id))
memset(&id, 0, sizeof(id));
initialized = 1;
return &id;
}

View File

@ -0,0 +1,87 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef __LIBCPUID_UTIL_H__
#define __LIBCPUID_UTIL_H__
#define COUNT_OF(array) (sizeof(array) / sizeof(array[0]))
struct feature_map_t {
unsigned bit;
cpu_feature_t feature;
};
void match_features(const struct feature_map_t* matchtable, int count,
uint32_t reg, struct cpu_id_t* data);
struct match_entry_t {
int family, model, stepping, ext_family, ext_model;
int ncores, l2cache, l3cache, brand_code, model_code;
char name[32];
};
void match_cpu_codename(const struct match_entry_t* matchtable, int count,
struct cpu_id_t* data, int brand_code, int model_code);
void warnf(const char* format, ...)
#ifdef __GNUC__
__attribute__((format(printf, 1, 2)))
#endif
;
void debugf(int verboselevel, const char* format, ...)
#ifdef __GNUC__
__attribute__((format(printf, 2, 3)))
#endif
;
void generic_get_cpu_list(const struct match_entry_t* matchtable, int count,
struct cpu_list_t* list);
/*
* Seek for a pattern in `haystack'.
* Pattern may be an fixed string, or contain the special metacharacters
* '.' - match any single character
* '#' - match any digit
* '[<chars>] - match any of the given chars (regex-like ranges are not
* supported)
* Return val: 0 if the pattern is not found. Nonzero if it is found (actually,
* x + 1 where x is the index where the match is found).
*/
int match_pattern(const char* haystack, const char* pattern);
/*
* Gets an initialized cpu_id_t. It is cached, so that internal libcpuid
* machinery doesn't need to issue cpu_identify more than once.
*/
struct cpu_id_t* get_cached_cpuid(void);
/*
* Sets the current errno
*/
int set_error(cpu_error_t err);
extern libcpuid_warn_fn_t _warn_fun;
extern int _current_verboselevel;
#endif /* __LIBCPUID_UTIL_H__ */

View File

@ -0,0 +1,295 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#include <stdio.h>
#include <string.h>
#include "libcpuid.h"
#include "libcpuid_util.h"
#include "asm-bits.h"
#include "rdtsc.h"
#ifdef _WIN32
#include <windows.h>
void sys_precise_clock(uint64_t *result)
{
double c, f;
LARGE_INTEGER freq, counter;
QueryPerformanceCounter(&counter);
QueryPerformanceFrequency(&freq);
c = (double) counter.QuadPart;
f = (double) freq.QuadPart;
*result = (uint64_t) ( c * 1000000.0 / f );
}
#else
/* assuming Linux, Mac OS or other POSIX */
#include <sys/time.h>
void sys_precise_clock(uint64_t *result)
{
struct timeval tv;
gettimeofday(&tv, NULL);
*result = (uint64_t) tv.tv_sec * (uint64_t) 1000000 +
(uint64_t) tv.tv_usec;
}
#endif /* _WIN32 */
/* out = a - b */
static void mark_t_subtract(struct cpu_mark_t* a, struct cpu_mark_t* b, struct cpu_mark_t *out)
{
out->tsc = a->tsc - b->tsc;
out->sys_clock = a->sys_clock - b->sys_clock;
}
void cpu_tsc_mark(struct cpu_mark_t* mark)
{
cpu_rdtsc(&mark->tsc);
sys_precise_clock(&mark->sys_clock);
}
void cpu_tsc_unmark(struct cpu_mark_t* mark)
{
struct cpu_mark_t temp;
cpu_tsc_mark(&temp);
mark_t_subtract(&temp, mark, mark);
}
int cpu_clock_by_mark(struct cpu_mark_t* mark)
{
uint64_t result;
/* Check if some subtraction resulted in a negative number: */
if ((mark->tsc >> 63) != 0 || (mark->sys_clock >> 63) != 0) return -1;
/* Divide-by-zero check: */
if (mark->sys_clock == 0) return -1;
/* Check if the result fits in 32bits */
result = mark->tsc / mark->sys_clock;
if (result > (uint64_t) 0x7fffffff) return -1;
return (int) result;
}
#ifdef _WIN32
int cpu_clock_by_os(void)
{
HKEY key;
DWORD result;
DWORD size = 4;
if (RegOpenKeyEx(HKEY_LOCAL_MACHINE, TEXT("HARDWARE\\DESCRIPTION\\System\\CentralProcessor\\0"), 0, KEY_READ, &key) != ERROR_SUCCESS)
return -1;
if (RegQueryValueEx(key, TEXT("~MHz"), NULL, NULL, (LPBYTE) &result, (LPDWORD) &size) != ERROR_SUCCESS) {
RegCloseKey(key);
return -1;
}
RegCloseKey(key);
return (int)result;
}
#else
#ifdef __APPLE__
#include <sys/types.h>
#include <sys/sysctl.h>
/* Assuming Mac OS X with hw.cpufrequency sysctl */
int cpu_clock_by_os(void)
{
long long result = -1;
size_t size = sizeof(result);
if (sysctlbyname("hw.cpufrequency", &result, &size, NULL, 0))
return -1;
return (int) (result / (long long) 1000000);
}
#else
/* Assuming Linux with /proc/cpuinfo */
int cpu_clock_by_os(void)
{
FILE *f;
char line[1024], *s;
int result;
f = fopen("/proc/cpuinfo", "rt");
if (!f) return -1;
while (fgets(line, sizeof(line), f)) {
if (!strncmp(line, "cpu MHz", 7)) {
s = strchr(line, ':');
if (s && 1 == sscanf(s, ":%d.", &result)) {
fclose(f);
return result;
}
}
}
fclose(f);
return -1;
}
#endif /* __APPLE__ */
#endif /* _WIN32 */
/* Emulate doing useful CPU intensive work */
static int busy_loop(int amount)
{
int i, j, k, s = 0;
static volatile int data[42] = {32, 12, -1, 5, 23, 0 };
for (i = 0; i < amount; i++)
for (j = 0; j < 65536; j++)
for (k = 0; k < 42; k++)
s += data[k];
return s;
}
int busy_loop_delay(int milliseconds)
{
int cycles = 0, r = 0, first = 1;
uint64_t a, b, c;
sys_precise_clock(&a);
while (1) {
sys_precise_clock(&c);
if ((c - a) / 1000 > milliseconds) return r;
r += busy_loop(cycles);
if (first) {
first = 0;
} else {
if (c - b < 1000) cycles *= 2;
if (c - b > 10000) cycles /= 2;
}
b = c;
}
}
int cpu_clock_measure(int millis, int quad_check)
{
struct cpu_mark_t begin[4], end[4], temp, temp2;
int results[4], cycles, n, k, i, j, bi, bj, mdiff, diff, _zero = 0;
uint64_t tl;
if (millis < 1) return -1;
tl = millis * (uint64_t) 1000;
if (quad_check)
tl /= 4;
n = quad_check ? 4 : 1;
cycles = 1;
for (k = 0; k < n; k++) {
cpu_tsc_mark(&begin[k]);
end[k] = begin[k];
do {
/* Run busy loop, and fool the compiler that we USE the garbishy
value it calculates */
_zero |= (1 & busy_loop(cycles));
cpu_tsc_mark(&temp);
mark_t_subtract(&temp, &end[k], &temp2);
/* If busy loop is too short, increase it */
if (temp2.sys_clock < tl / 8)
cycles *= 2;
end[k] = temp;
} while (end[k].sys_clock - begin[k].sys_clock < tl);
mark_t_subtract(&end[k], &begin[k], &temp);
results[k] = cpu_clock_by_mark(&temp);
}
if (n == 1) return results[0];
mdiff = 0x7fffffff;
bi = bj = -1;
for (i = 0; i < 4; i++) {
for (j = i + 1; j < 4; j++) {
diff = results[i] - results[j];
if (diff < 0) diff = -diff;
if (diff < mdiff) {
mdiff = diff;
bi = i;
bj = j;
}
}
}
if (results[bi] == -1) return -1;
return (results[bi] + results[bj] + _zero) / 2;
}
int cpu_clock_by_ic(int millis, int runs)
{
int max_value = 0, cur_value, i, ri, cycles_inner, cycles_outer, c;
struct cpu_id_t* id;
uint64_t t0, t1, tl, hz;
int multiplier_numerator = 1, multiplier_denom = 1;
if (millis <= 0 || runs <= 0) return -2;
id = get_cached_cpuid();
// if there aren't SSE instructions - we can't run the test at all
if (!id || !id->flags[CPU_FEATURE_SSE]) return -1;
//
if (id->sse_size < 128) {
debugf(1, "SSE execution path is 64-bit\n");
// on a CPU with half SSE unit length, SSE instructions execute at 0.5 IPC;
// the resulting value must be multiplied by 2:
multiplier_numerator = 2;
} else {
debugf(1, "SSE execution path is 128-bit\n");
}
//
// on a Bulldozer or later CPU, SSE instructions execute at 1.4 IPC, handle that as well:
if (id->vendor == VENDOR_AMD && id->ext_family >= 21) {
debugf(1, "cpu_clock_by_ic: Bulldozer (or later) detected, dividing result by 1.4\n");
multiplier_numerator = 5;
multiplier_denom = 7; // multiply by 5/7, to divide by 1.4
}
//
tl = millis * 125; // (*1000 / 8)
cycles_inner = 128;
cycles_outer = 1;
do {
if (cycles_inner < 1000000000) cycles_inner *= 2;
else cycles_outer *= 2;
sys_precise_clock(&t0);
for (i = 0; i < cycles_outer; i++)
busy_sse_loop(cycles_inner);
sys_precise_clock(&t1);
} while (t1 - t0 < tl);
debugf(2, "inner: %d, outer: %d\n", cycles_inner, cycles_outer);
for (ri = 0; ri < runs; ri++) {
sys_precise_clock(&t0);
c = 0;
do {
c++;
for (i = 0; i < cycles_outer; i++)
busy_sse_loop(cycles_inner);
sys_precise_clock(&t1);
} while (t1 - t0 < tl * (uint64_t) 8);
// cpu_Hz = cycles_inner * cycles_outer * 256 / (t1 - t0) * 1000000
debugf(2, "c = %d, td = %d\n", c, (int) (t1 - t0));
hz = ((uint64_t) cycles_inner * (uint64_t) 256 + 12) *
(uint64_t) cycles_outer * (uint64_t) multiplier_numerator * (uint64_t) c * (uint64_t) 1000000
/ ((t1 - t0) * (uint64_t) multiplier_denom);
cur_value = (int) (hz / 1000000);
if (cur_value > max_value) max_value = cur_value;
}
return max_value;
}
int cpu_clock(void)
{
int result;
result = cpu_clock_by_os();
if (result <= 0)
result = cpu_clock_measure(200, 1);
return result;
}

View File

@ -0,0 +1,33 @@
/*
* Copyright 2010 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef __RDTSC_H__
#define __RDTSC_H__
void sys_precise_clock(uint64_t *result);
int busy_loop_delay(int milliseconds);
#endif /* __RDTSC_H__ */

View File

@ -0,0 +1,469 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#include <stdio.h>
#include <string.h>
#include <ctype.h>
#include "libcpuid.h"
#include "recog_amd.h"
#include "libcpuid_util.h"
enum _amd_code_t {
NA,
NO_CODE,
OPTERON_GENERIC,
OPTERON_800,
ATHLON_XP,
ATHLON_XP_M,
ATHLON_XP_M_LV,
ATHLON,
ATHLON_MP,
MOBILE_ATHLON64,
ATHLON_FX,
DURON,
DURON_MP,
MOBILE_DURON,
MOBILE_SEMPRON,
OPTERON_SINGLE,
OPTERON_DUALCORE,
OPTERON_800_DUALCORE,
MOBILE_TURION,
ATHLON_64,
ATHLON_64_FX,
TURION_64,
TURION_X2,
SEMPRON,
M_SEMPRON,
SEMPRON_DUALCORE,
PHENOM,
PHENOM2,
ATHLON_64_X2,
ATHLON_64_X3,
ATHLON_64_X4,
FUSION_C,
FUSION_E,
FUSION_EA,
FUSION_Z,
};
typedef enum _amd_code_t amd_code_t;
const struct match_entry_t cpudb_amd[] = {
{ -1, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown AMD CPU" },
/* 486 and the likes */
{ 4, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown AMD 486" },
{ 4, 3, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "AMD 486DX2" },
{ 4, 7, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "AMD 486DX2WB" },
{ 4, 8, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "AMD 486DX4" },
{ 4, 9, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "AMD 486DX4WB" },
/* Pentia clones */
{ 5, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown AMD 586" },
{ 5, 0, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K5" },
{ 5, 1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K5" },
{ 5, 2, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K5" },
{ 5, 3, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K5" },
/* The K6 */
{ 5, 6, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K6" },
{ 5, 7, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K6" },
{ 5, 8, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K6-2" },
{ 5, 9, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K6-III" },
{ 5, 10, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown K6" },
{ 5, 11, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown K6" },
{ 5, 12, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown K6" },
{ 5, 13, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "K6-2+" },
/* Athlon et al. */
{ 6, 1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Athlon (Slot-A)" },
{ 6, 2, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Athlon (Slot-A)" },
{ 6, 3, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Duron (Spitfire)" },
{ 6, 4, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Athlon (ThunderBird)" },
{ 6, 6, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown Athlon" },
{ 6, 6, -1, -1, -1, 1, -1, -1, ATHLON , 0, "Athlon (Palomino)" },
{ 6, 6, -1, -1, -1, 1, -1, -1, ATHLON_MP , 0, "Athlon MP (Palomino)" },
{ 6, 6, -1, -1, -1, 1, -1, -1, DURON , 0, "Duron (Palomino)" },
{ 6, 6, -1, -1, -1, 1, -1, -1, ATHLON_XP , 0, "Athlon XP" },
{ 6, 7, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown Athlon XP" },
{ 6, 7, -1, -1, -1, 1, -1, -1, DURON , 0, "Duron (Morgan)" },
{ 6, 8, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Athlon XP" },
{ 6, 8, -1, -1, -1, 1, -1, -1, ATHLON , 0, "Athlon XP (Thoroughbred)" },
{ 6, 8, -1, -1, -1, 1, -1, -1, ATHLON_XP , 0, "Athlon XP (Thoroughbred)" },
{ 6, 8, -1, -1, -1, 1, -1, -1, DURON , 0, "Duron (Applebred)" },
{ 6, 8, -1, -1, -1, 1, -1, -1, SEMPRON , 0, "Sempron (Thoroughbred)" },
{ 6, 8, -1, -1, -1, 1, 128, -1, SEMPRON , 0, "Sempron (Thoroughbred)" },
{ 6, 8, -1, -1, -1, 1, 256, -1, SEMPRON , 0, "Sempron (Thoroughbred)" },
{ 6, 8, -1, -1, -1, 1, -1, -1, ATHLON_MP , 0, "Athlon MP (Thoroughbred)" },
{ 6, 8, -1, -1, -1, 1, -1, -1, ATHLON_XP_M , 0, "Mobile Athlon (T-Bred)" },
{ 6, 8, -1, -1, -1, 1, -1, -1, ATHLON_XP_M_LV , 0, "Mobile Athlon (T-Bred)" },
{ 6, 10, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Athlon XP (Barton)" },
{ 6, 10, -1, -1, -1, 1, 512, -1, ATHLON_XP , 0, "Athlon XP (Barton)" },
{ 6, 10, -1, -1, -1, 1, 512, -1, SEMPRON , 0, "Sempron (Barton)" },
{ 6, 10, -1, -1, -1, 1, 256, -1, SEMPRON , 0, "Sempron (Thorton)" },
{ 6, 10, -1, -1, -1, 1, 256, -1, ATHLON_XP , 0, "Athlon XP (Thorton)" },
{ 6, 10, -1, -1, -1, 1, -1, -1, ATHLON_MP , 0, "Athlon MP (Barton)" },
{ 6, 10, -1, -1, -1, 1, -1, -1, ATHLON_XP_M , 0, "Mobile Athlon (Barton)" },
{ 6, 10, -1, -1, -1, 1, -1, -1, ATHLON_XP_M_LV , 0, "Mobile Athlon (Barton)" },
/* K8 Architecture */
{ 15, -1, -1, 15, -1, 1, -1, -1, NO_CODE , 0, "Unknown K8" },
{ 15, -1, -1, 16, -1, 1, -1, -1, NO_CODE , 0, "Unknown K9" },
{ 15, -1, -1, 15, -1, 1, -1, -1, NO_CODE , 0, "Unknown A64" },
{ 15, -1, -1, 15, -1, 1, -1, -1, OPTERON_SINGLE , 0, "Opteron" },
{ 15, -1, -1, 15, -1, 2, -1, -1, OPTERON_DUALCORE , 0, "Opteron (Dual Core)" },
{ 15, 3, -1, 15, -1, 1, -1, -1, OPTERON_SINGLE , 0, "Opteron" },
{ 15, 3, -1, 15, -1, 2, -1, -1, OPTERON_DUALCORE , 0, "Opteron (Dual Core)" },
{ 15, -1, -1, 15, -1, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (512K)" },
{ 15, -1, -1, 15, -1, 1, 1024, -1, ATHLON_64 , 0, "Athlon 64 (1024K)" },
{ 15, -1, -1, 15, -1, 1, -1, -1, ATHLON_FX , 0, "Athlon FX" },
{ 15, -1, -1, 15, -1, 1, -1, -1, ATHLON_64_FX , 0, "Athlon 64 FX" },
{ 15, -1, -1, 15, -1, 2, 512, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (512K)" },
{ 15, -1, -1, 15, -1, 2, 1024, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (1024K)" },
{ 15, -1, -1, 15, -1, 1, 512, -1, TURION_64 , 0, "Turion 64 (512K)" },
{ 15, -1, -1, 15, -1, 1, 1024, -1, TURION_64 , 0, "Turion 64 (1024K)" },
{ 15, -1, -1, 15, -1, 2, 512, -1, TURION_X2 , 0, "Turion 64 X2 (512K)" },
{ 15, -1, -1, 15, -1, 2, 1024, -1, TURION_X2 , 0, "Turion 64 X2 (1024K)" },
{ 15, -1, -1, 15, -1, 1, 128, -1, SEMPRON , 0, "A64 Sempron (128K)" },
{ 15, -1, -1, 15, -1, 1, 256, -1, SEMPRON , 0, "A64 Sempron (256K)" },
{ 15, -1, -1, 15, -1, 1, 512, -1, SEMPRON , 0, "A64 Sempron (512K)" },
{ 15, -1, -1, 15, 0x4f, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (Orleans/512K)" },
{ 15, -1, -1, 15, 0x5f, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (Orleans/512K)" },
{ 15, -1, -1, 15, 0x2f, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (Venice/512K)" },
{ 15, -1, -1, 15, 0x2c, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (Venice/512K)" },
{ 15, -1, -1, 15, 0x1f, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (Winchester/512K)" },
{ 15, -1, -1, 15, 0x0c, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (Newcastle/512K)" },
{ 15, -1, -1, 15, 0x27, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (San Diego/512K)" },
{ 15, -1, -1, 15, 0x37, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (San Diego/512K)" },
{ 15, -1, -1, 15, 0x04, 1, 512, -1, ATHLON_64 , 0, "Athlon 64 (ClawHammer/512K)" },
{ 15, -1, -1, 15, 0x5f, 1, 1024, -1, ATHLON_64 , 0, "Athlon 64 (Orleans/1024K)" },
{ 15, -1, -1, 15, 0x27, 1, 1024, -1, ATHLON_64 , 0, "Athlon 64 (San Diego/1024K)" },
{ 15, -1, -1, 15, 0x04, 1, 1024, -1, ATHLON_64 , 0, "Athlon 64 (ClawHammer/1024K)" },
{ 15, -1, -1, 15, 0x4b, 2, 256, -1, SEMPRON_DUALCORE , 0, "Athlon 64 X2 (Windsor/256K)" },
{ 15, -1, -1, 15, 0x23, 2, 512, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (Toledo/512K)" },
{ 15, -1, -1, 15, 0x4b, 2, 512, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (Windsor/512K)" },
{ 15, -1, -1, 15, 0x43, 2, 512, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (Windsor/512K)" },
{ 15, -1, -1, 15, 0x6b, 2, 512, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (Brisbane/512K)" },
{ 15, -1, -1, 15, 0x2b, 2, 512, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (Manchester/512K)"},
{ 15, -1, -1, 15, 0x23, 2, 1024, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (Toledo/1024K)" },
{ 15, -1, -1, 15, 0x43, 2, 1024, -1, ATHLON_64_X2 , 0, "Athlon 64 X2 (Windsor/1024K)" },
{ 15, -1, -1, 15, 0x08, 1, 128, -1, M_SEMPRON , 0, "Mobile Sempron 64 (Dublin/128K)"},
{ 15, -1, -1, 15, 0x08, 1, 256, -1, M_SEMPRON , 0, "Mobile Sempron 64 (Dublin/256K)"},
{ 15, -1, -1, 15, 0x0c, 1, 256, -1, SEMPRON , 0, "Sempron 64 (Paris)" },
{ 15, -1, -1, 15, 0x1c, 1, 128, -1, SEMPRON , 0, "Sempron 64 (Palermo/128K)" },
{ 15, -1, -1, 15, 0x1c, 1, 256, -1, SEMPRON , 0, "Sempron 64 (Palermo/256K)" },
{ 15, -1, -1, 15, 0x1c, 1, 128, -1, M_SEMPRON , 0, "Mobile Sempron 64 (Sonora/128K)"},
{ 15, -1, -1, 15, 0x1c, 1, 256, -1, M_SEMPRON , 0, "Mobile Sempron 64 (Sonora/256K)"},
{ 15, -1, -1, 15, 0x2c, 1, 128, -1, SEMPRON , 0, "Sempron 64 (Palermo/128K)" },
{ 15, -1, -1, 15, 0x2c, 1, 256, -1, SEMPRON , 0, "Sempron 64 (Palermo/256K)" },
{ 15, -1, -1, 15, 0x2c, 1, 128, -1, M_SEMPRON , 0, "Mobile Sempron 64 (Albany/128K)"},
{ 15, -1, -1, 15, 0x2c, 1, 256, -1, M_SEMPRON , 0, "Mobile Sempron 64 (Albany/256K)"},
{ 15, -1, -1, 15, 0x2f, 1, 128, -1, SEMPRON , 0, "Sempron 64 (Palermo/128K)" },
{ 15, -1, -1, 15, 0x2f, 1, 256, -1, SEMPRON , 0, "Sempron 64 (Palermo/256K)" },
{ 15, -1, -1, 15, 0x4f, 1, 128, -1, SEMPRON , 0, "Sempron 64 (Manila/128K)" },
{ 15, -1, -1, 15, 0x4f, 1, 256, -1, SEMPRON , 0, "Sempron 64 (Manila/256K)" },
{ 15, -1, -1, 15, 0x5f, 1, 128, -1, SEMPRON , 0, "Sempron 64 (Manila/128K)" },
{ 15, -1, -1, 15, 0x5f, 1, 256, -1, SEMPRON , 0, "Sempron 64 (Manila/256K)" },
{ 15, -1, -1, 15, 0x6b, 2, 256, -1, SEMPRON , 0, "Sempron 64 Dual (Sherman/256K)"},
{ 15, -1, -1, 15, 0x6b, 2, 512, -1, SEMPRON , 0, "Sempron 64 Dual (Sherman/512K)"},
{ 15, -1, -1, 15, 0x7f, 1, 256, -1, SEMPRON , 0, "Sempron 64 (Sparta/256K)" },
{ 15, -1, -1, 15, 0x7f, 1, 512, -1, SEMPRON , 0, "Sempron 64 (Sparta/512K)" },
{ 15, -1, -1, 15, 0x4c, 1, 256, -1, M_SEMPRON , 0, "Mobile Sempron 64 (Keene/256K)"},
{ 15, -1, -1, 15, 0x4c, 1, 512, -1, M_SEMPRON , 0, "Mobile Sempron 64 (Keene/512K)"},
{ 15, -1, -1, 15, -1, 2, -1, -1, SEMPRON_DUALCORE , 0, "Sempron Dual Core" },
{ 15, -1, -1, 15, 0x24, 1, 512, -1, TURION_64 , 0, "Turion 64 (Lancaster/512K)" },
{ 15, -1, -1, 15, 0x24, 1, 1024, -1, TURION_64 , 0, "Turion 64 (Lancaster/1024K)" },
{ 15, -1, -1, 15, 0x48, 2, 256, -1, TURION_X2 , 0, "Turion X2 (Taylor)" },
{ 15, -1, -1, 15, 0x48, 2, 512, -1, TURION_X2 , 0, "Turion X2 (Trinidad)" },
{ 15, -1, -1, 15, 0x4c, 1, 512, -1, TURION_64 , 0, "Turion 64 (Richmond)" },
{ 15, -1, -1, 15, 0x68, 2, 256, -1, TURION_X2 , 0, "Turion X2 (Tyler/256K)" },
{ 15, -1, -1, 15, 0x68, 2, 512, -1, TURION_X2 , 0, "Turion X2 (Tyler/512K)" },
{ 15, -1, -1, 17, 3, 2, 512, -1, TURION_X2 , 0, "Turion X2 (Griffin/512K)" },
{ 15, -1, -1, 17, 3, 2, 1024, -1, TURION_X2 , 0, "Turion X2 (Griffin/1024K)" },
/* K9 Architecture */
{ 15, -1, -1, 16, -1, 1, -1, -1, PHENOM , 0, "Unknown AMD Phenom" },
{ 15, 2, -1, 16, -1, 1, -1, -1, PHENOM , 0, "Phenom" },
{ 15, 2, -1, 16, -1, 3, -1, -1, PHENOM , 0, "Phenom X3 (Toliman)" },
{ 15, 2, -1, 16, -1, 4, -1, -1, PHENOM , 0, "Phenom X4 (Agena)" },
{ 15, 2, -1, 16, -1, 3, 512, -1, PHENOM , 0, "Phenom X3 (Toliman/256K)" },
{ 15, 2, -1, 16, -1, 3, 512, -1, PHENOM , 0, "Phenom X3 (Toliman/512K)" },
{ 15, 2, -1, 16, -1, 4, 128, -1, PHENOM , 0, "Phenom X4 (Agena/128K)" },
{ 15, 2, -1, 16, -1, 4, 256, -1, PHENOM , 0, "Phenom X4 (Agena/256K)" },
{ 15, 2, -1, 16, -1, 4, 512, -1, PHENOM , 0, "Phenom X4 (Agena/512K)" },
{ 15, 2, -1, 16, -1, 2, 512, -1, ATHLON_64_X2 , 0, "Athlon X2 (Kuma)" },
/* Phenom II derivates: */
{ 15, 4, -1, 16, -1, 4, -1, -1, NO_CODE , 0, "Phenom (Deneb-based)" },
{ 15, 4, -1, 16, -1, 1, 1024, -1, SEMPRON , 0, "Sempron (Sargas)" },
{ 15, 4, -1, 16, -1, 2, 512, -1, PHENOM2 , 0, "Phenom II X2 (Callisto)" },
{ 15, 4, -1, 16, -1, 3, 512, -1, PHENOM2 , 0, "Phenom II X3 (Heka)" },
{ 15, 4, -1, 16, -1, 4, 512, -1, PHENOM2 , 0, "Phenom II X4" },
{ 15, 4, -1, 16, 4, 4, 512, -1, PHENOM2 , 0, "Phenom II X4 (Deneb)" },
{ 15, 5, -1, 16, 5, 4, 512, -1, PHENOM2 , 0, "Phenom II X4 (Deneb)" },
{ 15, 4, -1, 16, 10, 4, 512, -1, PHENOM2 , 0, "Phenom II X4 (Zosma)" },
{ 15, 4, -1, 16, 10, 6, 512, -1, PHENOM2 , 0, "Phenom II X6 (Thuban)" },
{ 15, 4, -1, 16, -1, 2, 1024, -1, ATHLON_64_X2 , 0, "Athlon II X2 (Regor)" },
{ 15, 4, -1, 16, -1, 2, 512, -1, ATHLON_64_X2 , 0, "Athlon II X2 (Regor)" },
{ 15, 5, -1, 16, 5, 3, 512, -1, ATHLON_64_X3 , 0, "Athlon II X3 (Rana)" },
{ 15, 5, -1, 16, 5, 4, 512, -1, ATHLON_64_X4 , 0, "Athlon II X4 (Propus)" },
/* 2011 CPUs with AMD fusion: */
{ 15, -1, -1, 20, 1, 1, 512, -1, FUSION_C , 0, "Brazos Ontario" },
{ 15, -1, -1, 20, 1, 2, 512, -1, FUSION_C , 0, "Brazos Ontario (Dual-core)" },
{ 15, -1, -1, 20, 1, 1, 512, -1, FUSION_E , 0, "Brazos Zacate" },
{ 15, -1, -1, 20, 1, 2, 512, -1, FUSION_E , 0, "Brazos Zacate (Dual-core)" },
{ 15, -1, -1, 20, 1, 1, 512, -1, FUSION_Z , 0, "Brazos Desna" },
{ 15, -1, -1, 18, 1, 2, 512, -1, FUSION_EA , 0, "Llano X2" },
{ 15, -1, -1, 18, 1, 2, 1024, -1, FUSION_EA , 0, "Llano X2" },
{ 15, -1, -1, 18, 1, 3, 1024, -1, FUSION_EA , 0, "Llano X3" },
{ 15, -1, -1, 18, 1, 4, 1024, -1, FUSION_EA , 0, "Llano X4" },
/* Newer Opterons: */
{ 15, 9, -1, 16, 9, 8, -1, -1, OPTERON_GENERIC , 0, "Magny-Cours Opteron" },
/* Bulldozer CPUs: */
{ 15, -1, -1, 21, 1, 4, 2048, -1, NO_CODE , 0, "Bulldozer X2" },
{ 15, -1, -1, 21, 1, 6, 2048, -1, NO_CODE , 0, "Bulldozer X3" },
{ 15, -1, -1, 21, 1, 8, 2048, -1, NO_CODE , 0, "Bulldozer X4" },
{ 15, -1, -1, 21, 2, 4, 2048, -1, NO_CODE , 0, "Vishera X2" },
{ 15, -1, -1, 21, 2, 6, 2048, -1, NO_CODE , 0, "Vishera X3" },
{ 15, -1, -1, 21, 2, 8, 2048, -1, NO_CODE , 0, "Vishera X4" },
};
static void load_amd_features(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
const struct feature_map_t matchtable_edx81[] = {
{ 20, CPU_FEATURE_NX },
{ 22, CPU_FEATURE_MMXEXT },
{ 25, CPU_FEATURE_FXSR_OPT },
{ 30, CPU_FEATURE_3DNOWEXT },
{ 31, CPU_FEATURE_3DNOW },
};
const struct feature_map_t matchtable_ecx81[] = {
{ 1, CPU_FEATURE_CMP_LEGACY },
{ 2, CPU_FEATURE_SVM },
{ 5, CPU_FEATURE_ABM },
{ 6, CPU_FEATURE_SSE4A },
{ 7, CPU_FEATURE_MISALIGNSSE },
{ 8, CPU_FEATURE_3DNOWPREFETCH },
{ 9, CPU_FEATURE_OSVW },
{ 10, CPU_FEATURE_IBS },
{ 11, CPU_FEATURE_XOP },
{ 12, CPU_FEATURE_SKINIT },
{ 13, CPU_FEATURE_WDT },
{ 16, CPU_FEATURE_FMA4 },
};
const struct feature_map_t matchtable_edx87[] = {
{ 0, CPU_FEATURE_TS },
{ 1, CPU_FEATURE_FID },
{ 2, CPU_FEATURE_VID },
{ 3, CPU_FEATURE_TTP },
{ 4, CPU_FEATURE_TM_AMD },
{ 5, CPU_FEATURE_STC },
{ 6, CPU_FEATURE_100MHZSTEPS },
{ 7, CPU_FEATURE_HWPSTATE },
/* id 8 is handled in common */
{ 9, CPU_FEATURE_CPB },
{ 10, CPU_FEATURE_APERFMPERF },
{ 11, CPU_FEATURE_PFI },
{ 12, CPU_FEATURE_PA },
};
if (raw->ext_cpuid[0][0] >= 0x80000001) {
match_features(matchtable_edx81, COUNT_OF(matchtable_edx81), raw->ext_cpuid[1][3], data);
match_features(matchtable_ecx81, COUNT_OF(matchtable_ecx81), raw->ext_cpuid[1][2], data);
}
if (raw->ext_cpuid[0][0] >= 0x80000001)
match_features(matchtable_edx87, COUNT_OF(matchtable_edx87), raw->ext_cpuid[7][3], data);
if (raw->ext_cpuid[0][0] >= 0x8000001a) {
/* We have the extended info about SSE unit size */
data->detection_hints[CPU_HINT_SSE_SIZE_AUTH] = 1;
data->sse_size = (raw->ext_cpuid[0x1a][0] & 1) ? 128 : 64;
}
}
static void decode_amd_cache_info(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
int l3_result;
const int assoc_table[16] = {
0, 1, 2, 0, 4, 0, 8, 0, 16, 0, 32, 48, 64, 92, 128, 255
};
unsigned n = raw->ext_cpuid[0][0];
if (n >= 0x80000005) {
data->l1_data_cache = (raw->ext_cpuid[5][2] >> 24) & 0xff;
data->l1_assoc = (raw->ext_cpuid[5][2] >> 16) & 0xff;
data->l1_cacheline = (raw->ext_cpuid[5][2]) & 0xff;
data->l1_instruction_cache = (raw->ext_cpuid[5][3] >> 24) & 0xff;
}
if (n >= 0x80000006) {
data->l2_cache = (raw->ext_cpuid[6][2] >> 16) & 0xffff;
data->l2_assoc = assoc_table[(raw->ext_cpuid[6][2] >> 12) & 0xf];
data->l2_cacheline = (raw->ext_cpuid[6][2]) & 0xff;
l3_result = (raw->ext_cpuid[6][3] >> 18);
if (l3_result > 0) {
l3_result = 512 * l3_result; /* AMD spec says it's a range,
but we take the lower bound */
data->l3_cache = l3_result;
data->l3_assoc = assoc_table[(raw->ext_cpuid[6][3] >> 12) & 0xf];
data->l3_cacheline = (raw->ext_cpuid[6][3]) & 0xff;
} else {
data->l3_cache = 0;
}
}
}
static void decode_amd_number_of_cores(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
int logical_cpus = -1, num_cores = -1;
if (raw->basic_cpuid[0][0] >= 1) {
logical_cpus = (raw->basic_cpuid[1][1] >> 16) & 0xff;
if (raw->ext_cpuid[0][0] >= 8) {
num_cores = 1 + (raw->ext_cpuid[8][2] & 0xff);
}
}
if (data->flags[CPU_FEATURE_HT]) {
if (num_cores > 1) {
data->num_cores = num_cores;
data->num_logical_cpus = logical_cpus;
} else {
data->num_cores = 1;
data->num_logical_cpus = (logical_cpus >= 2 ? logical_cpus : 2);
}
} else {
data->num_cores = data->num_logical_cpus = 1;
}
}
static int amd_has_turion_modelname(const char *bs)
{
/* We search for something like TL-60. Ahh, I miss regexes...*/
int i, l, k;
char code[3] = {0};
const char* codes[] = { "ML", "MT", "MK", "TK", "TL", "RM", "ZM", "" };
l = (int) strlen(bs);
for (i = 3; i < l - 2; i++) {
if (bs[i] == '-' &&
isupper(bs[i-1]) && isupper(bs[i-2]) && !isupper(bs[i-3]) &&
isdigit(bs[i+1]) && isdigit(bs[i+2]) && !isdigit(bs[i+3]))
{
code[0] = bs[i-2];
code[1] = bs[i-1];
for (k = 0; codes[k][0]; k++)
if (!strcmp(codes[k], code)) return 1;
}
}
return 0;
}
static amd_code_t decode_amd_codename_part1(const char *bs)
{
int is_dual = 0, is_quad = 0, is_tri = 0;
if (strstr(bs, "Dual Core") ||
strstr(bs, "Dual-Core") ||
strstr(bs, " X2 "))
is_dual = 1;
if (strstr(bs, " X4 ")) is_quad = 1;
if (strstr(bs, " X3 ")) is_tri = 1;
if (strstr(bs, "Opteron")) {
return is_dual ? OPTERON_DUALCORE : OPTERON_SINGLE;
}
if (strstr(bs, "Phenom")) {
if (strstr(bs, "II")) return PHENOM2;
else return PHENOM;
}
if (amd_has_turion_modelname(bs)) {
return is_dual ? TURION_X2 : TURION_64;
}
if (strstr(bs, "Athlon(tm) 64 FX")) return ATHLON_64_FX;
if (strstr(bs, "Athlon(tm) FX")) return ATHLON_FX;
if (strstr(bs, "Athlon(tm) 64") || strstr(bs, "Athlon(tm) II X") || match_pattern(bs, "Athlon(tm) X#")) {
if (is_quad) return ATHLON_64_X4;
if (is_dual) return ATHLON_64_X2;
if (is_tri) return ATHLON_64_X3;
return ATHLON_64;
}
if (strstr(bs, "Turion")) {
return is_dual ? TURION_X2 : TURION_64;
}
if (strstr(bs, "mobile") || strstr(bs, "Mobile")) {
if (strstr(bs, "Athlon(tm) XP-M (LV)")) return ATHLON_XP_M_LV;
if (strstr(bs, "Athlon(tm) XP")) return ATHLON_XP_M;
if (strstr(bs, "Sempron(tm)")) return M_SEMPRON;
if (strstr(bs, "Athlon")) return MOBILE_ATHLON64;
if (strstr(bs, "Duron")) return MOBILE_DURON;
} else {
if (strstr(bs, "Athlon(tm) XP")) return ATHLON_XP;
if (strstr(bs, "Athlon(tm) MP")) return ATHLON_MP;
if (strstr(bs, "Sempron(tm)")) return SEMPRON;
if (strstr(bs, "Duron")) return DURON;
if (strstr(bs, "Athlon")) return ATHLON;
}
if (match_pattern(bs, "C-##")) return FUSION_C;
if (match_pattern(bs, "E-###")) return FUSION_E;
if (match_pattern(bs, "Z-##")) return FUSION_Z;
if (match_pattern(bs, "E#-####") || match_pattern(bs, "A#-####")) return FUSION_EA;
return NO_CODE;
}
static void decode_amd_codename(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
amd_code_t code = decode_amd_codename_part1(data->brand_str);
if (code == ATHLON_64_X2 && data->l2_cache < 512)
code = SEMPRON_DUALCORE;
match_cpu_codename(cpudb_amd, COUNT_OF(cpudb_amd), data, code, 0);
}
int cpuid_identify_amd(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
load_amd_features(raw, data);
decode_amd_cache_info(raw, data);
decode_amd_number_of_cores(raw, data);
decode_amd_codename(raw, data);
return 0;
}
void cpuid_get_list_amd(struct cpu_list_t* list)
{
generic_get_cpu_list(cpudb_amd, COUNT_OF(cpudb_amd), list);
}

View File

@ -0,0 +1,32 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef __RECOG_AMD_H__
#define __RECOG_AMD_H__
int cpuid_identify_amd(struct cpu_raw_data_t* raw, struct cpu_id_t* data);
void cpuid_get_list_amd(struct cpu_list_t* list);
#endif /* __RECOG_AMD_H__ */

View File

@ -0,0 +1,791 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#include <string.h>
#include <ctype.h>
#include "libcpuid.h"
#include "recog_intel.h"
#include "libcpuid_util.h"
enum _intel_code_t {
NA,
NO_CODE,
PENTIUM = 10,
MOBILE_PENTIUM,
XEON = 20,
XEON_IRWIN,
XEONMP,
XEON_POTOMAC,
XEON_I7,
XEON_GAINESTOWN,
XEON_WESTMERE,
MOBILE_PENTIUM_M = 30,
CELERON,
MOBILE_CELERON,
NOT_CELERON,
CORE_SOLO = 40,
MOBILE_CORE_SOLO,
CORE_DUO,
MOBILE_CORE_DUO,
WOLFDALE = 50,
MEROM,
PENRYN,
QUAD_CORE,
DUAL_CORE_HT,
QUAD_CORE_HT,
MORE_THAN_QUADCORE,
PENTIUM_D,
ATOM = 60,
ATOM_SILVERTHORNE,
ATOM_DIAMONDVILLE,
ATOM_PINEVIEW,
ATOM_CEDARVIEW,
CORE_I3 = 70,
CORE_I5,
CORE_I7,
CORE_IVY3, /* 22nm Core-iX */
CORE_IVY5,
CORE_IVY7,
CORE_HASWELL3, /* 22nm Core-iX, Haswell */
CORE_HASWELL5,
CORE_HASWELL7,
};
typedef enum _intel_code_t intel_code_t;
enum _intel_model_t {
UNKNOWN = -1,
_3000 = 100,
_3100,
_3200,
X3200,
_3300,
X3300,
_5100,
_5200,
_5300,
_5400,
_2xxx, /* Core i[357] 2xxx */
_3xxx, /* Core i[357] 3xxx */
};
typedef enum _intel_model_t intel_model_t;
const struct match_entry_t cpudb_intel[] = {
{ -1, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown Intel CPU" },
/* i486 */
{ 4, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown i486" },
{ 4, 0, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 DX-25/33" },
{ 4, 1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 DX-50" },
{ 4, 2, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 SX" },
{ 4, 3, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 DX2" },
{ 4, 4, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 SL" },
{ 4, 5, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 SX2" },
{ 4, 7, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 DX2 WriteBack" },
{ 4, 8, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 DX4" },
{ 4, 9, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "i486 DX4 WriteBack" },
/* All Pentia:
Pentium 1 */
{ 5, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown Pentium" },
{ 5, 0, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium A-Step" },
{ 5, 1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium 1 (0.8u)" },
{ 5, 2, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium 1 (0.35u)" },
{ 5, 3, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium OverDrive" },
{ 5, 4, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium 1 (0.35u)" },
{ 5, 7, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium 1 (0.35u)" },
{ 5, 8, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium MMX (0.25u)" },
/* Pentium 2 / 3 / M / Conroe / whatsnext - all P6 based. */
{ 6, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown P6" },
{ 6, 0, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium Pro" },
{ 6, 1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium Pro" },
{ 6, 3, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium II (Klamath)" },
{ 6, 5, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium II (Deschutes)" },
{ 6, 5, -1, -1, -1, 1, -1, -1, MOBILE_PENTIUM , 0, "Mobile Pentium II (Tonga)"},
{ 6, 6, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium II (Dixon)" },
{ 6, 3, -1, -1, -1, 1, -1, -1, XEON , 0, "P-II Xeon" },
{ 6, 5, -1, -1, -1, 1, -1, -1, XEON , 0, "P-II Xeon" },
{ 6, 6, -1, -1, -1, 1, -1, -1, XEON , 0, "P-II Xeon" },
{ 6, 5, -1, -1, -1, 1, -1, -1, CELERON , 0, "P-II Celeron (no L2)" },
{ 6, 6, -1, -1, -1, 1, -1, -1, CELERON , 0, "P-II Celeron (128K)" },
/* -------------------------------------------------- */
{ 6, 7, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium III (Katmai)" },
{ 6, 8, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium III (Coppermine)"},
{ 6, 10, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium III (Coppermine)"},
{ 6, 11, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Pentium III (Tualatin)" },
{ 6, 7, -1, -1, -1, 1, -1, -1, XEON , 0, "P-III Xeon" },
{ 6, 8, -1, -1, -1, 1, -1, -1, XEON , 0, "P-III Xeon" },
{ 6, 10, -1, -1, -1, 1, -1, -1, XEON , 0, "P-III Xeon" },
{ 6, 11, -1, -1, -1, 1, -1, -1, XEON , 0, "P-III Xeon" },
{ 6, 7, -1, -1, -1, 1, -1, -1, CELERON , 0, "P-III Celeron" },
{ 6, 8, -1, -1, -1, 1, -1, -1, CELERON , 0, "P-III Celeron" },
{ 6, 10, -1, -1, -1, 1, -1, -1, CELERON , 0, "P-III Celeron" },
{ 6, 11, -1, -1, -1, 1, -1, -1, CELERON , 0, "P-III Celeron" },
/* Netburst based (Pentium 4 and later)
classic P4s */
{ 15, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown Pentium 4" },
{ 15, -1, -1, 15, -1, 1, -1, -1, CELERON , 0, "Unknown P-4 Celeron" },
{ 15, -1, -1, 15, -1, 1, -1, -1, XEON , 0, "Unknown Xeon" },
{ 15, 0, -1, 15, -1, 1, -1, -1, NO_CODE , 0, "Pentium 4 (Willamette)" },
{ 15, 1, -1, 15, -1, 1, -1, -1, NO_CODE , 0, "Pentium 4 (Willamette)" },
{ 15, 2, -1, 15, -1, 1, -1, -1, NO_CODE , 0, "Pentium 4 (Northwood)" },
{ 15, 3, -1, 15, -1, 1, -1, -1, NO_CODE , 0, "Pentium 4 (Prescott)" },
{ 15, 4, -1, 15, -1, 1, -1, -1, NO_CODE , 0, "Pentium 4 (Prescott)" },
{ 15, 6, -1, 15, -1, 1, -1, -1, NO_CODE , 0, "Pentium 4 (Cedar Mill)" },
{ 15, 0, -1, 15, -1, 1, -1, -1, MOBILE_PENTIUM , 0, "Mobile P-4 (Willamette)" },
{ 15, 1, -1, 15, -1, 1, -1, -1, MOBILE_PENTIUM , 0, "Mobile P-4 (Willamette)" },
{ 15, 2, -1, 15, -1, 1, -1, -1, MOBILE_PENTIUM , 0, "Mobile P-4 (Northwood)" },
{ 15, 3, -1, 15, -1, 1, -1, -1, MOBILE_PENTIUM , 0, "Mobile P-4 (Prescott)" },
{ 15, 4, -1, 15, -1, 1, -1, -1, MOBILE_PENTIUM , 0, "Mobile P-4 (Prescott)" },
{ 15, 6, -1, 15, -1, 1, -1, -1, MOBILE_PENTIUM , 0, "Mobile P-4 (Cedar Mill)" },
/* server CPUs */
{ 15, 0, -1, 15, -1, 1, -1, -1, XEON , 0, "Xeon (Foster)" },
{ 15, 1, -1, 15, -1, 1, -1, -1, XEON , 0, "Xeon (Foster)" },
{ 15, 2, -1, 15, -1, 1, -1, -1, XEON , 0, "Xeon (Prestonia)" },
{ 15, 2, -1, 15, -1, 1, -1, -1, XEONMP , 0, "Xeon (Gallatin)" },
{ 15, 3, -1, 15, -1, 1, -1, -1, XEON , 0, "Xeon (Nocona)" },
{ 15, 4, -1, 15, -1, 1, -1, -1, XEON , 0, "Xeon (Nocona)" },
{ 15, 4, -1, 15, -1, 1, -1, -1, XEON_IRWIN , 0, "Xeon (Irwindale)" },
{ 15, 4, -1, 15, -1, 1, -1, -1, XEONMP , 0, "Xeon (Cranford)" },
{ 15, 4, -1, 15, -1, 1, -1, -1, XEON_POTOMAC , 0, "Xeon (Potomac)" },
{ 15, 6, -1, 15, -1, 1, -1, -1, XEON , 0, "Xeon (Dempsey)" },
/* Pentium Ds */
{ 15, 4, 4, 15, -1, 1, -1, -1, NO_CODE , 0, "Pentium D" },
{ 15, 4, -1, 15, -1, 1, -1, -1, PENTIUM_D , 0, "Pentium D" },
{ 15, 4, 7, 15, -1, 1, -1, -1, NO_CODE , 0, "Pentium D" },
{ 15, 6, -1, 15, -1, 1, -1, -1, PENTIUM_D , 0, "Pentium D" },
/* Celeron and Celeron Ds */
{ 15, 1, -1, 15, -1, 1, -1, -1, CELERON , 0, "P-4 Celeron (128K)" },
{ 15, 2, -1, 15, -1, 1, -1, -1, CELERON , 0, "P-4 Celeron (128K)" },
{ 15, 3, -1, 15, -1, 1, -1, -1, CELERON , 0, "Celeron D" },
{ 15, 4, -1, 15, -1, 1, -1, -1, CELERON , 0, "Celeron D" },
{ 15, 6, -1, 15, -1, 1, -1, -1, CELERON , 0, "Celeron D" },
/* -------------------------------------------------- */
/* Intel Core microarchitecture - P6-based */
{ 6, 9, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown Pentium M" },
{ 6, 9, -1, -1, -1, 1, -1, -1, MOBILE_PENTIUM_M , 0, "Unknown Pentium M" },
{ 6, 9, -1, -1, -1, 1, -1, -1, PENTIUM , 0, "Pentium M (Banias)" },
{ 6, 9, -1, -1, -1, 1, -1, -1, MOBILE_PENTIUM_M , 0, "Pentium M (Banias)" },
{ 6, 9, -1, -1, -1, 1, -1, -1, CELERON , 0, "Celeron M" },
{ 6, 13, -1, -1, -1, 1, -1, -1, PENTIUM , 0, "Pentium M (Dothan)" },
{ 6, 13, -1, -1, -1, 1, -1, -1, MOBILE_PENTIUM_M , 0, "Pentium M (Dothan)" },
{ 6, 13, -1, -1, -1, 1, -1, -1, CELERON , 0, "Celeron M" },
{ 6, 12, -1, -1, -1, -1, -1, -1, ATOM , 0, "Unknown Atom" },
{ 6, 12, -1, -1, -1, -1, -1, -1, ATOM_DIAMONDVILLE , 0, "Atom (Diamondville)" },
{ 6, 12, -1, -1, -1, -1, -1, -1, ATOM_SILVERTHORNE , 0, "Atom (Silverthorne)" },
{ 6, 12, -1, -1, -1, -1, -1, -1, ATOM_CEDARVIEW , 0, "Atom (Cedarview)" },
{ 6, 6, -1, -1, -1, -1, -1, -1, ATOM_CEDARVIEW , 0, "Atom (Cedarview)" },
{ 6, 12, -1, -1, -1, -1, -1, -1, ATOM_PINEVIEW , 0, "Atom (Pineview)" },
/* -------------------------------------------------- */
{ 6, 14, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown Yonah" },
{ 6, 14, -1, -1, -1, 1, -1, -1, CORE_SOLO , 0, "Yonah (Core Solo)" },
{ 6, 14, -1, -1, -1, 2, -1, -1, CORE_DUO , 0, "Yonah (Core Duo)" },
{ 6, 14, -1, -1, -1, 1, -1, -1, MOBILE_CORE_SOLO , 0, "Yonah (Core Solo)" },
{ 6, 14, -1, -1, -1, 2, -1, -1, MOBILE_CORE_DUO , 0, "Yonah (Core Duo)" },
{ 6, 14, -1, -1, -1, 1, -1, -1, CORE_SOLO , 0, "Yonah (Core Solo)" },
{ 6, 15, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Unknown Core 2" },
{ 6, 15, -1, -1, -1, 2, 4096, -1, CORE_DUO , 0, "Conroe (Core 2 Duo)" },
{ 6, 15, -1, -1, -1, 2, 1024, -1, CORE_DUO , 0, "Conroe (Core 2 Duo) 1024K" },
{ 6, 15, -1, -1, -1, 2, 512, -1, CORE_DUO , 0, "Conroe (Core 2 Duo) 512K" },
{ 6, 15, -1, -1, -1, 4, -1, -1, QUAD_CORE , 0, "Kentsfield (Core 2 Quad)" },
{ 6, 15, -1, -1, -1, 4, 4096, -1, QUAD_CORE , 0, "Kentsfield (Core 2 Quad)" },
{ 6, 15, -1, -1, -1, 400, -1, -1, MORE_THAN_QUADCORE, 0, "More than quad-core" },
{ 6, 15, -1, -1, -1, 2, 2048, -1, CORE_DUO , 0, "Allendale (Core 2 Duo)" },
{ 6, 15, -1, -1, -1, 2, -1, -1, MOBILE_CORE_DUO , 0, "Merom (Core 2 Duo)" },
{ 6, 15, -1, -1, -1, 2, 2048, -1, MEROM , 0, "Merom (Core 2 Duo) 2048K" },
{ 6, 15, -1, -1, -1, 2, 4096, -1, MEROM , 0, "Merom (Core 2 Duo) 4096K" },
{ 6, 15, -1, -1, 15, 1, -1, -1, CELERON , 0, "Conroe-L (Celeron)" },
{ 6, 6, -1, -1, 22, 1, -1, -1, CELERON , 0, "Conroe-L (Celeron)" },
{ 6, 15, -1, -1, 15, 2, -1, -1, CELERON , 0, "Conroe-L (Allendale)" },
{ 6, 6, -1, -1, 22, 2, -1, -1, CELERON , 0, "Conroe-L (Allendale)" },
{ 6, 6, -1, -1, 22, 1, -1, -1, NO_CODE , 0, "Unknown Core ?" },
{ 6, 7, -1, -1, 23, 1, -1, -1, NO_CODE , 0, "Unknown Core ?" },
{ 6, 6, -1, -1, 22, 400, -1, -1, MORE_THAN_QUADCORE, 0, "More than quad-core" },
{ 6, 7, -1, -1, 23, 400, -1, -1, MORE_THAN_QUADCORE, 0, "More than quad-core" },
{ 6, 7, -1, -1, 23, 1, -1, -1, CORE_SOLO , 0, "Unknown Core 45nm" },
{ 6, 7, -1, -1, 23, 1, -1, -1, CORE_DUO , 0, "Unknown Core 45nm" },
{ 6, 7, -1, -1, 23, 2, 1024, -1, WOLFDALE , 0, "Celeron Wolfdale 1M" },
{ 6, 7, -1, -1, 23, 2, 2048, -1, WOLFDALE , 0, "Wolfdale (Core 2 Duo) 2M" },
{ 6, 7, -1, -1, 23, 2, 3072, -1, WOLFDALE , 0, "Wolfdale (Core 2 Duo) 3M" },
{ 6, 7, -1, -1, 23, 2, 6144, -1, WOLFDALE , 0, "Wolfdale (Core 2 Duo) 6M" },
{ 6, 7, -1, -1, 23, 1, -1, -1, MOBILE_CORE_DUO , 0, "Penryn (Core 2 Duo)" },
{ 6, 7, -1, -1, 23, 2, 3072, -1, PENRYN , 0, "Penryn (Core 2 Duo) 3M" },
{ 6, 7, -1, -1, 23, 2, 6144, -1, PENRYN , 0, "Penryn (Core 2 Duo) 6M" },
{ 6, 7, -1, -1, 23, 4, 2048, -1, QUAD_CORE , 0, "Yorkfield (Core 2 Quad) 2M"},
{ 6, 7, -1, -1, 23, 4, 3072, -1, QUAD_CORE , 0, "Yorkfield (Core 2 Quad) 3M"},
{ 6, 7, -1, -1, 23, 4, 6144, -1, QUAD_CORE , 0, "Yorkfield (Core 2 Quad) 6M"},
{ 6, 5, -1, -1, 37, 2, -1, -1, NO_CODE , 0, "Unknown Core i3/i5 CPU" },
{ 6, 5, -1, -1, 37, 2, -1, 4096, CORE_I7 , 0, "Arrandale (Core i7)" },
{ 6, 5, -1, -1, 37, 2, -1, 3072, CORE_I5 , 0, "Arrandale (Core i5)" },
{ 6, 5, -1, -1, 37, 2, -1, 4096, CORE_I5 , 0, "Clarkdale (Core i5)" },
{ 6, 5, -1, -1, 37, 4, -1, 8192, CORE_I5 , 0, "Lynnfield (Core i5)" },
{ 6, 5, -1, -1, 37, 2, -1, 3072, CORE_I3 , 0, "Arrandale (Core i3)" },
{ 6, 5, -1, -1, 37, 2, -1, 4096, CORE_I3 , 0, "Clarkdale (Core i3)" },
{ 6, 10, -1, -1, 42, -1, -1, -1, NO_CODE , 0, "Unknown Sandy Bridge" },
{ 6, 10, -1, -1, 42, -1, -1, -1, CORE_I7 , 0, "Sandy Bridge i7" },
{ 6, 10, -1, -1, 42, 4, -1, -1, CORE_I7 , 0, "Sandy Bridge (Core i7)" },
{ 6, 10, -1, -1, 42, 4, -1, -1, CORE_I5 , 0, "Sandy Bridge (Core i5)" },
{ 6, 10, -1, -1, 42, 2, -1, -1, CORE_I3 , 0, "Sandy Bridge (Core i3)" },
{ 6, 10, -1, -1, 42, 1, -1, -1, CELERON , 0, "Celeron (Sandy Bridge)" },
{ 6, 10, -1, -1, 42, 2, -1, -1, CELERON , 0, "Celeron (Sandy Bridge)" },
{ 6, 10, -1, -1, 42, 2, -1, -1, PENTIUM , 0, "Pentium (Sandy Bridge)" },
{ 6, 10, -1, -1, 26, 1, -1, -1, CORE_I7 , 0, "Intel Core i7" },
{ 6, 10, -1, -1, 26, 4, -1, -1, CORE_I7 , 0, "Bloomfield (Core i7)" },
{ 6, 10, -1, -1, 30, 4, -1, -1, CORE_I7 , 0, "Lynnfield (Core i7)" },
{ 6, 10, -1, -1, 26, 4, -1, -1, XEON_I7 , 0, "Xeon (Bloomfield)" },
{ 6, 10, -1, -1, 26, 4, -1, -1, XEON_GAINESTOWN , 0, "Xeon (Gainestown)" },
{ 6, 10, -1, -1, 26, 4, -1, 4096, XEON_GAINESTOWN , 0, "Xeon (Gainestown) 4M" },
{ 6, 10, -1, -1, 26, 4, -1, 8192, XEON_GAINESTOWN , 0, "Xeon (Gainestown) 8M" },
{ 6, 12, -1, -1, 44, -1, -1, -1, XEON_WESTMERE , 0, "Xeon (Westmere-based)" },
{ 6, 12, -1, -1, 44, 4, -1, 12288, CORE_I7 , 0, "Gulftown (Core i7)" },
{ 6, 12, -1, -1, 44, -1, -1, 12288, XEON_WESTMERE , 0, "Xeon (Gulftown)" },
{ 6, 13, -1, -1, 45, -1, -1, -1, XEON , 0, "Xeon (Sandy Bridge)" },
{ 6, 13, -1, -1, 45, -1, -1, -1, CORE_I7 , 0, "Sandy Bridge-E (Core i7)" },
{ 6, 13, -1, -1, 45, -1, -1, -1, CORE_I5 , 0, "Sandy Bridge-E (Core i5)" },
{ 6, 13, -1, -1, 45, -1, -1, -1, CORE_I3 , 0, "Sandy Bridge-E (Core i3)" },
{ 6, 10, -1, -1, 58, 4, -1, -1, CORE_IVY7 , 0, "Ivy Bridge (Core i7)" },
{ 6, 10, -1, -1, 58, 4, -1, -1, CORE_IVY5 , 0, "Ivy Bridge (Core i5)" },
{ 6, 10, -1, -1, 58, 2, -1, -1, CORE_IVY3 , 0, "Ivy Bridge (Core i3)" },
{ 6, 12, -1, -1, 60, 4, -1, -1, CORE_HASWELL7 , 0, "Haswell (Core i7)" },
{ 6, 12, -1, -1, 60, 4, -1, -1, CORE_HASWELL5 , 0, "Haswell (Core i5)" },
{ 6, 12, -1, -1, 60, 2, -1, -1, CORE_HASWELL3 , 0, "Haswell (Core i3)" },
/* Core microarchitecture-based Xeons: */
{ 6, 14, -1, -1, 14, 1, -1, -1, XEON , 0, "Xeon LV" },
{ 6, 15, -1, -1, 15, 2, 4096, -1, XEON , _5100, "Xeon (Woodcrest)" },
{ 6, 15, -1, -1, 15, 2, 2048, -1, XEON , _3000, "Xeon (Conroe/2M)" },
{ 6, 15, -1, -1, 15, 2, 4096, -1, XEON , _3000, "Xeon (Conroe/4M)" },
{ 6, 15, -1, -1, 15, 4, 4096, -1, XEON , X3200, "Xeon (Kentsfield)" },
{ 6, 15, -1, -1, 15, 4, 4096, -1, XEON , _5300, "Xeon (Clovertown)" },
{ 6, 7, -1, -1, 23, 2, 6144, -1, XEON , _3100, "Xeon (Wolfdale)" },
{ 6, 7, -1, -1, 23, 2, 6144, -1, XEON , _5200, "Xeon (Wolfdale DP)" },
{ 6, 7, -1, -1, 23, 4, 6144, -1, XEON , _5400, "Xeon (Harpertown)" },
{ 6, 7, -1, -1, 23, 4, 3072, -1, XEON , X3300, "Xeon (Yorkfield/3M)" },
{ 6, 7, -1, -1, 23, 4, 6144, -1, XEON , X3300, "Xeon (Yorkfield/6M)" },
/* Itaniums */
{ 7, -1, -1, -1, -1, 1, -1, -1, NO_CODE , 0, "Itanium" },
{ 15, -1, -1, 16, -1, 1, -1, -1, NO_CODE , 0, "Itanium 2" },
};
static void load_intel_features(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
const struct feature_map_t matchtable_edx1[] = {
{ 18, CPU_FEATURE_PN },
{ 21, CPU_FEATURE_DTS },
{ 22, CPU_FEATURE_ACPI },
{ 27, CPU_FEATURE_SS },
{ 29, CPU_FEATURE_TM },
{ 30, CPU_FEATURE_IA64 },
{ 31, CPU_FEATURE_PBE },
};
const struct feature_map_t matchtable_ecx1[] = {
{ 1, CPU_FEATURE_PCLMUL },
{ 2, CPU_FEATURE_DTS64 },
{ 4, CPU_FEATURE_DS_CPL },
{ 5, CPU_FEATURE_VMX },
{ 6, CPU_FEATURE_SMX },
{ 7, CPU_FEATURE_EST },
{ 8, CPU_FEATURE_TM2 },
{ 10, CPU_FEATURE_CID },
{ 14, CPU_FEATURE_XTPR },
{ 15, CPU_FEATURE_PDCM },
{ 18, CPU_FEATURE_DCA },
{ 20, CPU_FEATURE_SSE4_2 },
{ 22, CPU_FEATURE_MOVBE },
{ 25, CPU_FEATURE_AES },
{ 26, CPU_FEATURE_XSAVE },
{ 27, CPU_FEATURE_OSXSAVE },
{ 28, CPU_FEATURE_AVX },
{ 30, CPU_FEATURE_RDRAND },
};
const struct feature_map_t matchtable_ebx7[] = {
{ 5, CPU_FEATURE_AVX2 },
};
const struct feature_map_t matchtable_edx81[] = {
{ 20, CPU_FEATURE_XD },
};
if (raw->basic_cpuid[0][0] >= 1) {
match_features(matchtable_edx1, COUNT_OF(matchtable_edx1), raw->basic_cpuid[1][3], data);
match_features(matchtable_ecx1, COUNT_OF(matchtable_ecx1), raw->basic_cpuid[1][2], data);
}
if (raw->basic_cpuid[0][0] >= 7) {
match_features(matchtable_ebx7, COUNT_OF(matchtable_ebx7), raw->basic_cpuid[7][1], data);
}
if (raw->ext_cpuid[0][0] >= 1) {
match_features(matchtable_edx81, COUNT_OF(matchtable_edx81), raw->ext_cpuid[1][3], data);
}
}
enum _cache_type_t {
L1I,
L1D,
L2,
L3
};
typedef enum _cache_type_t cache_type_t;
static void check_case(uint8_t on, cache_type_t cache, int size, int assoc, int linesize, struct cpu_id_t* data)
{
if (!on) return;
switch (cache) {
case L1I:
data->l1_instruction_cache = size;
break;
case L1D:
data->l1_data_cache = size;
data->l1_assoc = assoc;
data->l1_cacheline = linesize;
break;
case L2:
data->l2_cache = size;
data->l2_assoc = assoc;
data->l2_cacheline = linesize;
break;
case L3:
data->l3_cache = size;
data->l3_assoc = assoc;
data->l3_cacheline = linesize;
default:
break;
}
}
static void decode_intel_oldstyle_cache_info(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
uint8_t f[256] = {0};
int reg, off;
uint32_t x;
for (reg = 0; reg < 4; reg++) {
x = raw->basic_cpuid[2][reg];
if (x & 0x80000000) continue;
for (off = 0; off < 4; off++) {
f[x & 0xff] = 1;
x >>= 8;
}
}
check_case(f[0x06], L1I, 8, 4, 32, data);
check_case(f[0x08], L1I, 16, 4, 32, data);
check_case(f[0x0A], L1D, 8, 2, 32, data);
check_case(f[0x0C], L1D, 16, 4, 32, data);
check_case(f[0x22], L3, 512, 4, 64, data);
check_case(f[0x23], L3, 1024, 8, 64, data);
check_case(f[0x25], L3, 2048, 8, 64, data);
check_case(f[0x29], L3, 4096, 8, 64, data);
check_case(f[0x2C], L1D, 32, 8, 64, data);
check_case(f[0x30], L1I, 32, 8, 64, data);
check_case(f[0x39], L2, 128, 4, 64, data);
check_case(f[0x3A], L2, 192, 6, 64, data);
check_case(f[0x3B], L2, 128, 2, 64, data);
check_case(f[0x3C], L2, 256, 4, 64, data);
check_case(f[0x3D], L2, 384, 6, 64, data);
check_case(f[0x3E], L2, 512, 4, 64, data);
check_case(f[0x41], L2, 128, 4, 32, data);
check_case(f[0x42], L2, 256, 4, 32, data);
check_case(f[0x43], L2, 512, 4, 32, data);
check_case(f[0x44], L2, 1024, 4, 32, data);
check_case(f[0x45], L2, 2048, 4, 32, data);
check_case(f[0x46], L3, 4096, 4, 64, data);
check_case(f[0x47], L3, 8192, 8, 64, data);
check_case(f[0x4A], L3, 6144, 12, 64, data);
check_case(f[0x4B], L3, 8192, 16, 64, data);
check_case(f[0x4C], L3, 12288, 12, 64, data);
check_case(f[0x4D], L3, 16384, 16, 64, data);
check_case(f[0x4E], L2, 6144, 24, 64, data);
check_case(f[0x60], L1D, 16, 8, 64, data);
check_case(f[0x66], L1D, 8, 4, 64, data);
check_case(f[0x67], L1D, 16, 4, 64, data);
check_case(f[0x68], L1D, 32, 4, 64, data);
/* The following four entries are trace cache. Intel does not
* specify a cache-line size, so we use -1 instead
*/
check_case(f[0x70], L1I, 12, 8, -1, data);
check_case(f[0x71], L1I, 16, 8, -1, data);
check_case(f[0x72], L1I, 32, 8, -1, data);
check_case(f[0x73], L1I, 64, 8, -1, data);
check_case(f[0x78], L2, 1024, 4, 64, data);
check_case(f[0x79], L2, 128, 8, 64, data);
check_case(f[0x7A], L2, 256, 8, 64, data);
check_case(f[0x7B], L2, 512, 8, 64, data);
check_case(f[0x7C], L2, 1024, 8, 64, data);
check_case(f[0x7D], L2, 2048, 8, 64, data);
check_case(f[0x7F], L2, 512, 2, 64, data);
check_case(f[0x82], L2, 256, 8, 32, data);
check_case(f[0x83], L2, 512, 8, 32, data);
check_case(f[0x84], L2, 1024, 8, 32, data);
check_case(f[0x85], L2, 2048, 8, 32, data);
check_case(f[0x86], L2, 512, 4, 64, data);
check_case(f[0x87], L2, 1024, 8, 64, data);
if (f[0x49]) {
/* This flag is overloaded with two meanings. On Xeon MP
* (family 0xf, model 0x6) this means L3 cache. On all other
* CPUs (notably Conroe et al), this is L2 cache. In both cases
* it means 4MB, 16-way associative, 64-byte line size.
*/
if (data->family == 0xf && data->model == 0x6) {
data->l3_cache = 4096;
data->l3_assoc = 16;
data->l3_cacheline = 64;
} else {
data->l2_cache = 4096;
data->l2_assoc = 16;
data->l2_cacheline = 64;
}
}
if (f[0x40]) {
/* Again, a special flag. It means:
* 1) If no L2 is specified, then CPU is w/o L2 (0 KB)
* 2) If L2 is specified by other flags, then, CPU is w/o L3.
*/
if (data->l2_cache == -1) {
data->l2_cache = 0;
} else {
data->l3_cache = 0;
}
}
}
static void decode_intel_deterministic_cache_info(struct cpu_raw_data_t* raw,
struct cpu_id_t* data)
{
int ecx;
int ways, partitions, linesize, sets, size, level, typenumber;
cache_type_t type;
for (ecx = 0; ecx < MAX_INTELFN4_LEVEL; ecx++) {
typenumber = raw->intel_fn4[ecx][0] & 0x1f;
if (typenumber == 0) break;
level = (raw->intel_fn4[ecx][0] >> 5) & 0x7;
if (level == 1 && typenumber == 1)
type = L1D;
else if (level == 1 && typenumber == 2)
type = L1I;
else if (level == 2 && typenumber == 3)
type = L2;
else if (level == 3 && typenumber == 3)
type = L3;
else {
warnf("deterministic_cache: unknown level/typenumber combo (%d/%d), cannot\n", level, typenumber);
warnf("deterministic_cache: recognize cache type\n");
continue;
}
ways = ((raw->intel_fn4[ecx][1] >> 22) & 0x3ff) + 1;
partitions = ((raw->intel_fn4[ecx][1] >> 12) & 0x3ff) + 1;
linesize = (raw->intel_fn4[ecx][1] & 0xfff) + 1;
sets = raw->intel_fn4[ecx][2] + 1;
size = ways * partitions * linesize * sets / 1024;
check_case(1, type, size, ways, linesize, data);
}
}
static int decode_intel_extended_topology(struct cpu_raw_data_t* raw,
struct cpu_id_t* data)
{
int i, level_type, num_smt = -1, num_core = -1;
for (i = 0; i < MAX_INTELFN11_LEVEL; i++) {
level_type = (raw->intel_fn11[i][2] & 0xff00) >> 8;
switch (level_type) {
case 0x01:
num_smt = raw->intel_fn11[i][1] & 0xffff;
break;
case 0x02:
num_core = raw->intel_fn11[i][1] & 0xffff;
break;
default:
break;
}
}
if (num_smt == -1 || num_core == -1) return 0;
data->num_cores = num_core / num_smt;
data->num_logical_cpus = num_core;
return 1;
}
static void decode_intel_number_of_cores(struct cpu_raw_data_t* raw,
struct cpu_id_t* data)
{
int logical_cpus = -1, num_cores = -1;
if (raw->basic_cpuid[0][0] >= 11) {
if (decode_intel_extended_topology(raw, data)) return;
}
if (raw->basic_cpuid[0][0] >= 1) {
logical_cpus = (raw->basic_cpuid[1][1] >> 16) & 0xff;
if (raw->basic_cpuid[0][0] >= 4) {
num_cores = 1 + ((raw->basic_cpuid[4][0] >> 26) & 0x3f);
}
}
if (data->flags[CPU_FEATURE_HT]) {
if (num_cores > 1) {
data->num_cores = num_cores;
data->num_logical_cpus = logical_cpus;
} else {
data->num_cores = 1;
data->num_logical_cpus = (logical_cpus >= 2 ? logical_cpus : 2);
}
} else {
data->num_cores = data->num_logical_cpus = 1;
}
}
static intel_code_t get_brand_code(struct cpu_id_t* data)
{
intel_code_t code = NO_CODE;
int i, need_matchtable = 1, core_ix_base = 0;
const char* bs = data->brand_str;
const char* s;
const struct { intel_code_t c; const char *search; } matchtable[] = {
{ XEONMP, "Xeon MP" },
{ XEONMP, "Xeon(TM) MP" },
{ XEON, "Xeon" },
{ CELERON, "Celeron" },
{ MOBILE_PENTIUM_M, "Pentium(R) M" },
{ CORE_SOLO, "Pentium(R) Dual CPU" },
{ PENTIUM_D, "Pentium(R) D" },
{ PENTIUM, "Pentium" },
{ CORE_SOLO, "Genuine Intel(R) CPU" },
{ CORE_SOLO, "Intel(R) Core(TM)" },
{ ATOM_DIAMONDVILLE, "Atom(TM) CPU [N ][23]## " },
{ ATOM_SILVERTHORNE, "Atom(TM) CPU Z" },
{ ATOM_PINEVIEW, "Atom(TM) CPU D" },
{ ATOM_CEDARVIEW, "Atom(TM) CPU N####" },
{ ATOM, "Atom(TM) CPU" },
};
if (strstr(bs, "Mobile")) {
need_matchtable = 0;
if (strstr(bs, "Celeron"))
code = MOBILE_CELERON;
else if (strstr(bs, "Pentium"))
code = MOBILE_PENTIUM;
}
if ((i = match_pattern(bs, "Core(TM) i[357]")) != 0) {
/* Core i3, Core i5 or Core i7 */
need_matchtable = 0;
core_ix_base = CORE_I3;
/* if it has RdRand, then it is at least Ivy Bridge */
if (data->flags[CPU_FEATURE_RDRAND])
core_ix_base = CORE_IVY3;
/* if it has FMA, then it is at least Haswell */
if (data->flags[CPU_FEATURE_FMA3])
core_ix_base = CORE_HASWELL3;
switch (bs[i + 9]) {
case '3': code = core_ix_base + 0; break;
case '5': code = core_ix_base + 1; break;
case '7': code = core_ix_base + 2; break;
}
}
if (need_matchtable) {
for (i = 0; i < COUNT_OF(matchtable); i++)
if (match_pattern(bs, matchtable[i].search)) {
code = matchtable[i].c;
break;
}
debugf(2, "intel matchtable result is %d\n", code);
}
if (code == XEON) {
if (match_pattern(bs, "W35##") || match_pattern(bs, "[ELXW]75##"))
code = XEON_I7;
else if (match_pattern(bs, "[ELXW]55##"))
code = XEON_GAINESTOWN;
else if (match_pattern(bs, "[ELXW]56##"))
code = XEON_WESTMERE;
else if (data->l3_cache > 0)
code = XEON_IRWIN;
}
if (code == XEONMP && data->l3_cache > 0)
code = XEON_POTOMAC;
if (code == CORE_SOLO) {
s = strstr(bs, "CPU");
if (s) {
s += 3;
while (*s == ' ') s++;
if (*s == 'T')
code = (data->num_cores == 1) ? MOBILE_CORE_SOLO : MOBILE_CORE_DUO;
}
}
if (code == CORE_SOLO) {
switch (data->num_cores) {
case 1: break;
case 2:
{
code = CORE_DUO;
if (data->num_logical_cpus > 2)
code = DUAL_CORE_HT;
break;
}
case 4:
{
code = QUAD_CORE;
if (data->num_logical_cpus > 4)
code = QUAD_CORE_HT;
break;
}
default:
code = MORE_THAN_QUADCORE; break;
}
}
if (code == CORE_DUO && data->ext_model >= 23) {
code = WOLFDALE;
}
if (code == PENTIUM_D && data->ext_model >= 23) {
code = WOLFDALE;
}
if (code == MOBILE_CORE_DUO && data->model != 14) {
if (data->ext_model < 23) {
code = MEROM;
} else {
code = PENRYN;
}
}
return code;
}
static intel_model_t get_model_code(struct cpu_id_t* data)
{
int i = 0;
int l = (int) strlen(data->brand_str);
const char *bs = data->brand_str;
int mod_flags = 0, model_no = 0, ndigs = 0;
/* If the CPU is a Core ix, then just return the model number generation: */
if ((i = match_pattern(bs, "Core(TM) i[357]")) != 0) {
i += 11;
if (i + 4 >= l) return UNKNOWN;
if (bs[i] == '2') return _2xxx;
if (bs[i] == '3') return _3xxx;
return UNKNOWN;
}
/* For Core2-based Xeons: */
while (i < l - 3) {
if (bs[i] == 'C' && bs[i+1] == 'P' && bs[i+2] == 'U')
break;
i++;
}
if (i >= l - 3) return UNKNOWN;
i += 3;
while (i < l - 4 && bs[i] == ' ') i++;
if (i >= l - 4) return UNKNOWN;
while (i < l - 4 && !isdigit(bs[i])) {
if (bs[i] >= 'A' && bs[i] <= 'Z')
mod_flags |= (1 << (bs[i] - 'A'));
i++;
}
if (i >= l - 4) return UNKNOWN;
while (isdigit(bs[i])) {
ndigs++;
model_no = model_no * 10 + (int) (bs[i] - '0');
i++;
}
if (ndigs != 4) return UNKNOWN;
#define HAVE(ch, flags) ((flags & (1 << ((int)(ch-'A')))) != 0)
switch (model_no / 100) {
case 30: return _3000;
case 31: return _3100;
case 32:
{
return (HAVE('X', mod_flags)) ? X3200 : _3200;
}
case 33:
{
return (HAVE('X', mod_flags)) ? X3300 : _3300;
}
case 51: return _5100;
case 52: return _5200;
case 53: return _5300;
case 54: return _5400;
default:
return UNKNOWN;
}
#undef HAVE
}
int cpuid_identify_intel(struct cpu_raw_data_t* raw, struct cpu_id_t* data)
{
load_intel_features(raw, data);
if (raw->basic_cpuid[0][0] >= 4) {
/* Deterministic way is preferred, being more generic */
decode_intel_deterministic_cache_info(raw, data);
} else if (raw->basic_cpuid[0][0] >= 2) {
decode_intel_oldstyle_cache_info(raw, data);
}
decode_intel_number_of_cores(raw, data);
match_cpu_codename(cpudb_intel, COUNT_OF(cpudb_intel), data,
get_brand_code(data), get_model_code(data));
return 0;
}
void cpuid_get_list_intel(struct cpu_list_t* list)
{
generic_get_cpu_list(cpudb_intel, COUNT_OF(cpudb_intel), list);
}

View File

@ -0,0 +1,32 @@
/*
* Copyright 2008 Veselin Georgiev,
* anrieffNOSPAM @ mgail_DOT.com (convert to gmail)
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
*
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef __RECOG_INTEL_H__
#define __RECOG_INTEL_H__
int cpuid_identify_intel(struct cpu_raw_data_t* raw, struct cpu_id_t* data);
void cpuid_get_list_intel(struct cpu_list_t* list);
#endif /*__RECOG_INTEL_H__*/