blob: 92b713ff1ff11836526a2965544547367b5d60c0 [file] [log] [blame]
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
= System Views
WARNING: The system views are an experimental feature and can be changed in future releases.
Ignite provides a number of built-in SQL views that contain information about cluster nodes and node metrics.
The views are available in the SYS schema.
See the link:SQL/schemas[Understanding Schemas] page for the information on how to access a non-default schema.
[IMPORTANT]
====
[discrete]
=== Limitations
. You cannot create objects in the SYS schema.
. System views from the SYS schema cannot be joined with user tables.
====
== Querying System Views
To query the system views using the link:sqlline[SQLLine] tool, connect to the SYS schema as follows:
[source, shell]
----
./sqlline.sh -u jdbc:ignite:thin://127.0.0.1/SYS
----
If your node is running on a remote server, replace `127.0.0.1` with the IP address of the server.
Run a query:
[source, sql]
----
-- get the list of nodes
select * from NODES;
-- view the CPU load as a percentage for a specific node
select CUR_CPU_LOAD * 100 from NODE_METRICS where NODE_ID = 'a1b77663-b37f-4ddf-87a6-1e2d684f3bae'
----
The same example using link:thin-clients/java-thin-client[Java Thin Client]:
[source, java]
----
include::{javaCodeDir}/JavaThinClient.java[tag=system-views,indent=0]
----
:table_opts: cols="2,1,4",opts="header"
== CACHES
[{table_opts}]
|===
| Column | Type | Description
|CACHE_NAME | string | Cache name
|CACHE_ID | int | Cache ID
|CACHE_TYPE | string | Cache type
|CACHE_MODE | string | Cache mode
|ATOMICITY_MODE | string | Atomicity mode
|CACHE_GROUP_NAME | string | Cache group name
|AFFINITY | string | toString representation of affinity function
|AFFINITY_MAPPER | string | toString representation of affinity mapper
|BACKUPS | int | backup count
|CACHE_GROUP_ID | int | cache group id
|CACHE_LOADER_FACTORY | string | toString representation of cache loader factory
|CACHE_STORE_FACTORY | string | toString representation of cache store factory
|CACHE_WRITER_FACTORY | string | toString representation of cache writer factory
|DATA_REGION_NAME | string | Data region name
|DEFAULT_LOCK_TIMEOUT | long | Lock timeout in milliseconds
|EVICTION_FILTER | string | toString representation of eviction filter
|EVICTION_POLICY_FACTORY | string | toString representation of eviction policy factory
|EXPIRY_POLICY_FACTORY | string | toString representation of expiry policy factory
|INTERCEPTOR | string | toString representation of interceptor
|IS_COPY_ON_READ | boolean | Flag indicating whether a copy of the value stored in the on-heap cache
|IS_EAGER_TTL | boolean | Flag indicating whether expired cache entries will be eagerly removed from cache
|IS_ENCRYPTION_ENABLED | boolean | True if cache data encrypted
|IS_EVENTS_DISABLED | boolean | True if events disabled for this cache
|IS_INVALIDATE | boolean | True if values will be invalidated (nullified) upon commit in near cache
|IS_LOAD_PREVIOUS_VALUE | boolean | True if value should be loaded from store if it is not in the cache
|IS_MANAGEMENT_ENABLED | boolean|
|IS_NEAR_CACHE_ENABLED | boolean| True if near cache enabled
|IS_ONHEAP_CACHE_ENABLED | boolean | True if on heap cache enabled
|IS_READ_FROM_BACKUP | boolean | True if read operation should be performed from backup node
|IS_READ_THROUGH | boolean | True if read from third party storage enabled
|IS_SQL_ESCAPE_ALL | boolean | If true all the SQL table and field names will be escaped with double quotes
|IS_SQL_ONHEAP_CACHE_ENABLED | boolean | If true SQL on-heap cache is enabled. When enabled, Ignite will cache SQL rows as they are accessed by query engine. Rows are invalidated and evicted from cache when relevant cache entry is either changed or evicted.
|IS_STATISTICS_ENABLED | boolean|
|IS_STORE_KEEP_BINARY | boolean| Flag indicating that {@link CacheStore} implementation is working with binary objects instead of Java objects.
|IS_WRITE_BEHIND_ENABLED | boolean | Flag indicating whether Ignite should use write-behind behaviour for the cache store
|IS_WRITE_THROUGH | boolean | True if write to third party storage enabled
|MAX_CONCURRENT_ASYNC_OPERATIONS | int | Maximum number of allowed concurrent asynchronous operations. If 0 returned then number of concurrent asynchronous operations is unlimited
|MAX_QUERY_ITERATORS_COUNT | int | Maximum number of query iterators that can be stored. Iterators are stored to support query pagination when each page of data is sent to user's node only on demand
|NEAR_CACHE_EVICTION_POLICY_FACTORY | string | toString representation of near cache eviction policy factory
|NEAR_CACHE_START_SIZE | int | Initial cache size for near cache which will be used to pre-create internal hash table after start.
|NODE_FILTER | string | toString representation of node filter
|PARTITION_LOSS_POLICY | string | toString representation of partition loss policy
|QUERY_DETAIL_METRICS_SIZE | int | size of queries detail metrics that will be stored in memory for monitoring purposes. If 0 then history will not be collected.
|QUERY_PARALLELISM | int | Hint to query execution engine on desired degree of parallelism within a single node
|REBALANCE_BATCH_SIZE | int | Size (in bytes) to be loaded within a single rebalance message
|REBALANCE_BATCHES_PREFETCH_COUNT | int | Number of batches generated by supply node at rebalancing start
|REBALANCE_DELAY | long | Rebalance delay in milliseconds
|REBALANCE_MODE | string | Rebalance mode
|REBALANCE_ORDER | int | Rebalance order
|REBALANCE_THROTTLE | long | Time in milliseconds to wait between rebalance messages to avoid overloading of CPU or network
|REBALANCE_TIMEOUT | long | Rebalance timeout in milliseconds
|SQL_INDEX_MAX_INLINE_SIZE | int | Index inline size in bytes
|SQL_ONHEAP_CACHE_MAX_SIZE | int | Maximum SQL on-heap cache. Measured in number of rows. When maximum size is reached oldest cached rows will be evicted.
|SQL_SCHEMA | string | Schema name
|TOPOLOGY_VALIDATOR | string | toString representation of topology validator
|WRITE_BEHIND_BATCH_SIZE | int | Maximum batch size for write-behind cache store operations
|WRITE_BEHIND_COALESCING | boolean | Write coalescing flag for write-behind cache store operations. Store operations (get or remove) with the same key are combined or coalesced to single, resulting operation to reduce pressure to underlying cache store
|WRITE_BEHIND_FLUSH_FREQUENCY | long | Frequency with which write-behind cache is flushed to the cache store in milliseconds
|WRITE_BEHIND_FLUSH_SIZE | int | Maximum size of the write-behind cache. If cache size exceeds this value, all cached items are flushed to the cache store and write cache is cleared
|WRITE_BEHIND_FLUSH_THREAD_COUNT | int | Number of threads that will perform cache flushing
|WRITE_SYNCHRONIZATION_MODE | string | Gets write synchronization mode
|===
== CACHE_GROUPS
The CACHE_GROUPS view contains information about the link:configuring-caches/cache-groups[cache groups].
[{table_opts}]
|===
|Column|Data Type|Description
|AFFINITY| VARCHAR | The string representation (as returned by the `toString()` method) of the affinity function defined for the cache group.
|ATOMICITY_MODE | VARCHAR | The link:configuring-caches/atomicity-modes[atomicity mode] of the cache group.
|BACKUPS|INT | The number of link:configuring-caches/configuring-backups[backup partitions] configured for the cache group.
|CACHE_COUNT|INT | The number of caches in the cache group.
|CACHE_GROUP_ID|INT | The ID of the cache group.
|CACHE_GROUP_NAME | VARCHAR | The name of the cache group.
|CACHE_MODE | VARCHAR | The cache mode.
|DATA_REGION_NAME | VARCHAR | The name of the link:memory-configuration/data-regions[data region].
|IS_SHARED|BOOLEAN | If this group contains more than one cache.
|NODE_FILTER | VARCHAR | The string representation (as returned by the `toString()` method) of the node filter defined for the cache group.
|PARTITION_LOSS_POLICY | VARCHAR | link:configuring-caches/partition-loss-policy[Partition loss policy].
|PARTITIONS_COUNT|INT | The number of partitions.
|REBALANCE_DELAY|LONG | link:data-rebalancing#other-properties[Rebalancing delay].
|REBALANCE_MODE | VARCHAR | link:data-rebalancing#configuring-rebalancing-mode[Rebalancing mode].
|REBALANCE_ORDER|INT | link:data-rebalancing#other-properties[Rebalancing order].
|TOPOLOGY_VALIDATOR | VARCHAR | The string representation (as returned by the `toString()` method) of the topology validator defined for the cache group.
|===
== TASKS
This view exposes information about currently running compute tasks started by a node. For instance, let's assume that an
application started a compute task using the Ignite thick client and the task's job was executed on one of the server nodes.
In this case, the thick client will report statistics related to the task via this system view while the server node will
be updating the thick client with task-related execution details.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|ID | UUID | Task id
|SESSION_ID | UUID | Session ID
|TASK_NODE_ID | UUID | Task originating node id
|TASK_NAME | string | Task name
|TASK_CLASS_NAME | string | Task class name
|AFFINITY_PARTITION_ID | int | Cache partition id
|AFFINITY_CACHE_NAME | string | Cache name
|START_TIME | long | Start time
|END_TIME | long | End time
|EXEC_NAME | string | Thread pool name executing task
|INTERNAL | boolean | True if task is internal
|USER_VERSION | string | Task user version
|===
== JOBS
This system view shows a list of compute jobs started by a node as part of a compute task.
To view the status of the compute task refer to the `TASKS` system view.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|ID | UUID | Job ID
|SESSION_ID | UUID | Job's session ID. Note, `SESSION_ID` is equal to `TASKS.SESSION_ID` for the jobs belonging to a specific task.
|ORIGIN_NODE_ID | UUID | The id of the node that started the job
|TASK_NAME | string | The name of the task
|TASK_CLASSNAME | string | Class name of the task
|AFFINITY_CACHE_IDS | string | IDs of one or more caches if the job is executed against one of the `IgniteCompute.affinity..`
methods. The parameter is empty, if you use `IgniteCompute` APIs that don't target specific caches.
|AFFINITY_PARTITION_ID | int | IDs of one or more partitions if the job is executed via one of the `IgniteCompute.affinity..`
methods. The parameter is empty, if you use `IgniteCompute` APIs that don't target specific partitions.
|CREATE_TIME | long | Job's creation time
|START_TIME | long | Job's start time
|FINISH_TIME | long | Job's finish time
|EXECUTOR_NAME | string | The name of the task's executor
|IS_FINISHING | boolean | `True` if the job is finishing
|IS_INTERNAL | boolean | `True` if the job is internal
|IS_STARTED | boolean | `True` if the job has been started
|IS_TIMEDOUT | boolean | `True` if the job timed out before completing
|STATE | string | Possible values: +
`ACTIVE` - Job is being executed. +
`PASSIVE` - Job is added to the execution queue. Please, see `CollisionSPI` for more details. +
`CANCELED` - Job is canceled.
|===
== SERVICES
[{table_opts}]
|===
|AFFINITY_KEY | string | Affinity key value for service
|CACHE_NAME | string | Cache name
|MAX_PER_NODE_COUNT | int | Maximum count of services instances per node
|NAME | string | Service name
|NAME | TYPE | DESCRIPTION
|NODE_FILTER | string | toString representation of node filter
|ORIGIN_NODE_ID | UUID | Originating node ID
|SERVICE_CLASS | string | Service class name
|SERVICE_ID | UUID | Service ID
|STATICALLY_CONFIGURED | boolean | True is service statically configured
|TOTAL_COUNT | int | Total count of service instances
|===
== TRANSACTIONS
This view exposes information about currently running transactions.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|ORIGINATING_NODE_ID | UUID |
|STATE | string |
|XID | UUID |
|LABEL | string |
|START_TIME | long |
|ISOLATION | string |
|CONCURRENCY | string |
|KEYS_COUNT | int |
|CACHE_IDS | string |
|COLOCATED | boolean |
|DHT | boolean |
|DURATION | long |
|IMPLICIT | boolean |
|IMPLICIT_SINGLE | boolean |
|INTERNAL | boolean |
|LOCAL | boolean |
|LOCAL_NODE_ID | UUID |
|NEAR | boolean |
|ONE_PHASE_COMMIT | boolean |
|OTHER_NODE_ID | UUID |
|SUBJECT_ID | UUID |
|SYSTEM | boolean |
|THREAD_ID | long |
|TIMEOUT | long |
|TOP_VER | string |
|===
== NODES
The NODES view contains information about the cluster nodes.
[cols="1,1,2",opts="header"]
|===
| Column | Data Type |Description
| IS_LOCAL| BOOLEAN| Whether the node is local.
|ADDRESSES |VARCHAR |The addresses of the node.
|CONSISTENT_ID |VARCHAR |Node's consistent ID.
|HOSTNAMES |VARCHAR |The host names of the node.
|IS_CLIENT |BOOLEAN |Indicates whether the node is a client.
|IS_DAEMON |BOOLEAN |Indicates whether the node is a daemon node.
|NODE_ID |UUID |Node ID.
|NODE_ORDER |INT |Node order within the topology.
|VERSION |VARCHAR |Node version.
|===
== NODE_ATTRIBUTES
The NODE_ATTRIBUTES view contains the attributes of all nodes.
[{table_opts}]
|===
| Column |Data Type |Description
|NODE_ID |UUID |Node ID.
|NAME |VARCHAR |Attribute name.
|===
== BASELINE_NODES
The BASELINE_NODES view contains information about the nodes that are part of the current baseline topology.
[{table_opts}]
|===
| Column |Data Type |Description
|CONSISTENT_ID |VARCHAR |Node consistent ID.
|ONLINE |BOOLEAN |Indicates whether the node is up and running.
|===
== CLIENT_CONNECTIONS
This view exposes information about currently opened client connections: JDBC, ODBC, Thin clients.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|CONNECTION_ID | long | ID of the connection
|LOCAL_ADDRESS | IP | address IP address of the local node
|REMOTE_ADDRESS | IP | address IP address of the remote node
|TYPE | string | Type of the connection
|USER | string | User name
|VERSION | string | Protocol version
|===
== STRIPED_THREADPOOL_QUEUE
This view exposes information about tasks waiting for the execution in the system striped thread pool.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|DESCRIPTION | string | toString representation of the task
|STRIPE_INDEX | int | Index of the stripe thread
|TASK_NAME | string | Class name of the task
|THREAD_NAME | string | Name of the stripe thread
|===
== DATASTREAM_THREADPOOL_QUEUE
This view exposes information about tasks waiting for the execution in the data streamer stripped thread pool.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|DESCRIPTION | string | toString representation of the task
|STRIPE_INDEX | int | Index of the stripe thread
|TASK_NAME | string | Class name of the task
|THREAD_NAME | string | Name of the stripe thread
|===
== SCAN_QUERIES
This view exposes information about currently running scan queries.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|CACHE_GROUP_ID | int | Cache group ID
|CACHE_GROUP_NAME | string | Cache group name
|CACHE_ID | int | Cache ID
|CACHE_NAME | string | Cache name
|CANCELED | boolean | True if canceled
|DURATION | long | Query duration
|FILTER | string | toString representation of filter
|KEEP_BINARY | boolean | True if keepBinary enabled
|LOCAL | boolean | True if query local only
|ORIGIN_NODE_ID | UUID | Node id started query
|PAGE_SIZE | int | Page size
|PARTITION | int | Query partition ID
|QUERY_ID | long | Query ID
|START_TIME | long | Query start time
|SUBJECT_ID | UUID | User ID started query
|TASK_NAME | string |
|TOPOLOGY | string | Topology version
|TRANSFORMER | string | toString representation of transformer
|===
== CONTINUOUS_QUERIES
This view exposes information about currently running continuous queries.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|AUTO_UNSUBSCRIBE | boolean | True if query should be stopped when node disconnected or originating node left
|BUFFER_SIZE | int | Event batch buffer size
|CACHE_NAME | string | Cache name
|DELAYED_REGISTER | boolean | True if query would be started when corresponding cache started
|INTERVAL | long | Notify interval
|IS_EVENTS | boolean | True if used for subscription to remote events
|IS_MESSAGING | boolean | True if used for subscription to messages.
|IS_QUERY | boolean | True if user started continuous query.
|KEEP_BINARY | boolean | True if keepBinary enabled
|LAST_SEND_TIME | long | Last time event batch sent to query originating node
|LOCAL_LISTENER | string | toString representation of local listener
|LOCAL_TRANSFORMED_LISTENER | string | toString representation of local transformed listener
|NODE_ID | UUID | Originating node id
|NOTIFY_EXISTING | boolean | True if listener should be notified about existing entries
|OLD_VALUE_REQUIRED | boolean | True if old entry value should be included in event
|REMOTE_FILTER | string | toString representation of remote filter
|REMOTE_TRANSFORMER | string | toString representation of remote transformer
|ROUTINE_ID | UUID | Query ID
|TOPIC | string | Query topic name
|===
== SQL_QUERIES
This view exposes information about currently running SQL queries.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|DURATION | long | Query execution duration
|LOCAL | boolean | True if local only
|ORIGIN_NODE_ID | UUID | Node that started query
|QUERY_ID | UUID | Query ID
|SCHEMA_NAME | string | Schema name
|SQL | string | Query text
|START_TIME | date | Query start time
|===
== SQL_QUERIES_HISTORY
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|SCHEMA_NAME | string | Schema name
|SQL | string | Query text
|LOCAL | boolean | True if local only
|EXECUTIONS | long | Count of executions
|FAILURES | long | Count of failures
|DURATION_MIN | long | Minimal duration of execution
|DURATION_MAX | long | Maximum duration of execution
|LAST_START_TIME | date | Last execution date
|===
== SCHEMAS
This view exposes information about SQL schemas.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|NAME | string| Name of the schema
|PREDEFINED | boolean | If true schema is predefined
|===
== NODE_METRICS
The NODE_METRICS view provides various metrics about the state of nodes, resource consumption and other metrics.
[cols="2,1,4",opts="header,stretch"]
|===
|Column|Data Type|Description
|NODE_ID|UUID| Node ID.
|LAST_UPDATE_TIME|TIMESTAMP|Last time the metrics were updated.
|MAX_ACTIVE_JOBS|INT| Maximum number of concurrent jobs this node ever had at one time.
|CUR_ACTIVE_JOBS|INT| Number of currently active jobs running on the node.
|AVG_ACTIVE_JOBS|FLOAT| Average number of active jobs concurrently executing on the node.
|MAX_WAITING_JOBS|INT|Maximum number of waiting jobs this node ever had at one time.
|CUR_WAITING_JOBS|INT|Number of queued jobs currently waiting to be executed.
|AVG_WAITING_JOBS|FLOAT| Average number of waiting jobs this node ever had at one time.
|MAX_REJECTED_JOBS|INT| Maximum number of jobs rejected at once during a single collision resolution operation.
|CUR_REJECTED_JOBS|INT|Number of jobs rejected as a result of the most recent collision resolution operation.
|AVG_REJECTED_JOBS|FLOAT| Average number of jobs this node rejected as a result of collision resolution operations.
|TOTAL_REJECTED_JOBS|INT| Total number of jobs this node has rejected as a result of collision resolution operations since the node startup.
|MAX_CANCELED_JOBS|INT| Maximum number of cancelled jobs this node ever had running concurrently.
|CUR_CANCELED_JOBS|INT| Number of cancelled jobs that are still running.
|AVG_CANCELED_JOBS|FLOAT| Average number of cancelled jobs this node ever had running concurrently.
|TOTAL_CANCELED_JOBS|INT| Number of jobs cancelled since the node startup.
|MAX_JOBS_WAIT_TIME|TIME| Maximum time a job ever spent waiting in a queue before being executed.
|CUR_JOBS_WAIT_TIME|TIME| Longest wait time among the jobs that are currently waiting for execution.
|AVG_JOBS_WAIT_TIME|TIME| Average time jobs spend in the queue before being executed.
|MAX_JOBS_EXECUTE_TIME|TIME| Maximum job execution time.
|CUR_JOBS_EXECUTE_TIME|TIME| Longest time a current job has been executing for.
|AVG_JOBS_EXECUTE_TIME|TIME| Average job execution time on this node.
|TOTAL_JOBS_EXECUTE_TIME|TIME|Total time all finished jobs took to execute on this node since the node startup.
|TOTAL_EXECUTED_JOBS|INT| Total number of jobs handled by the node since the node startup.
|TOTAL_EXECUTED_TASKS|INT| Total number of tasks handled by the node.
|TOTAL_BUSY_TIME|TIME| Total time this node spent executing jobs.
|TOTAL_IDLE_TIME|TIME| Total time this node spent idling (not executing any jobs).
|CUR_IDLE_TIME|TIME| Time this node has spent idling since executing the last job.
|BUSY_TIME_PERCENTAGE|FLOAT|Percentage of job execution vs idle time.
|IDLE_TIME_PERCENTAGE|FLOAT|Percentage of idle vs job execution time.
|TOTAL_CPU|INT| Number of CPUs available to the Java Virtual Machine.
|CUR_CPU_LOAD|DOUBLE| Percentage of CPU usage expressed as a fraction in the range [0, 1].
|AVG_CPU_LOAD|DOUBLE| Average percentage of CPU usage expressed as a fraction in the range [0, 1].
|CUR_GC_CPU_LOAD|DOUBLE| Average time spent in GC since the last update of the metrics. By default, metrics are updated every 2 seconds.
|HEAP_MEMORY_INIT|LONG| Amount of heap memory in bytes that the JVM initially requests from the operating system for memory management. Shows `-1` if the initial memory size is undefined.
|HEAP_MEMORY_USED|LONG| Current heap size that is used for object allocation. The heap consists of one or more memory pools. This value is the sum of used heap memory values of all heap memory pools.
|HEAP_MEMORY_COMMITED|LONG| Amount of heap memory in bytes that is committed for the JVM to use. This amount of memory is guaranteed for the JVM to use. The heap consists of one or more memory pools. This value is the sum of committed heap memory values of all heap memory pools.
|HEAP_MEMORY_MAX|LONG| Maximum amount of heap memory in bytes that can be used for memory management. The column displays `-1` if the maximum memory size is undefined.
|HEAP_MEMORY_TOTAL|LONG| Total amount of heap memory in bytes. The column displays `-1` if the total memory size is undefined.
|NONHEAP_MEMORY_INIT|LONG| Amount of non-heap memory in bytes that the JVM initially requests from the operating system for memory management. The column displays `-1` if the initial memory size is undefined.
|NONHEAP_MEMORY_USED|LONG| Current non-heap memory size that is used by Java VM. The non-heap memory consists of one or more memory pools. This value is the sum of used non-heap memory values of all non-heap memory pools.
|NONHEAP_MEMORY_COMMITED|LONG| Amount of non-heap memory in bytes that is committed for the JVM to use. This amount of memory is guaranteed for the JVM to use. The non-heap memory consists of one or more memory pools. This value is the sum of committed non-heap memory values of all non-heap memory pools.
|NONHEAP_MEMORY_MAX|LONG| Returns the maximum amount of non-heap memory in bytes that can be used for memory management. The column displays `-1` if the maximum memory size is undefined.
|NONHEAP_MEMORY_TOTAL|LONG| Total amount of non-heap memory in bytes that can be used for memory management. The column displays `-1` if the total memory size is undefined.
|UPTIME|TIME|Uptime of the JVM.
|JVM_START_TIME|TIMESTAMP|Start time of the JVM.
|NODE_START_TIME|TIMESTAMP| Start time of the node.
|LAST_DATA_VERSION|LONG|In-Memory Data Grid assigns incremental versions to all cache operations. This column contains the latest data version on the node.
|CUR_THREAD_COUNT|INT| Number of live threads including both daemon and non-daemon threads.
|MAX_THREAD_COUNT|INT| Maximum live thread count since the JVM started or peak was reset.
|TOTAL_THREAD_COUNT|LONG| Total number of threads started since the JVM started.
|CUR_DAEMON_THREAD_COUNT|INT|Number of live daemon threads.
|SENT_MESSAGES_COUNT|INT|Number of node communication messages sent.
|SENT_BYTES_COUNT|LONG| Amount of bytes sent.
|RECEIVED_MESSAGES_COUNT|INT|Number of node communication messages received.
|RECEIVED_BYTES_COUNT|LONG| Amount of bytes received.
|OUTBOUND_MESSAGES_QUEUE|INT| Outbound messages queue size.
|===
== TABLES
The TABLES view contains information about the SQL tables.
[{table_opts}]
|===
|Column|Data Type|Description
|AFFINITY_KEY_COLUMN | string | Affinity key column name
|CACHE_ID | int | Cache id for the table
|CACHE_NAME | string | Cache name for the table
|IS_INDEX_REBUILD_IN_PROGRESS | boolean | True if some index rebuild for this table in progress
|KEY_ALIAS | string | Key column alias
|KEY_TYPE_NAME | string | Key type name
|SCHEMA_NAME | string | Schema name of the table
|TABLE_NAME | string | Name of the table
|VALUE_ALIAS | string | Value column alias
|VALUE_TYPE_NAME | string | Value type name
|===
== TABLE_COLUMNS
This view exposes information about SQL table columns.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|AFFINITY_COLUMN | boolean | True if column affinity key.
|AUTO_INCREMENT | boolean | True if auto incremented
|COLUMN_NAME | string | Column name
|DEFAULT_VALUE | string | Default column value
|NULLABLE | boolean | True if nullable
|PK | boolean | True if primary key
|PRECISION | int | Column precision
|SCALE | int | Column scale
|SCHEMA_NAME | string | Schema name
|TABLE_NAME | string | Table name
|TYPE | string | Column type
|===
== VIEWS
This view exposes information about SQL views.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|NAME | string | Name
|SCHEMA | string | Schema
|DESCRIPTION | string | Description
|===
== VIEW_COLUMNS
This view exposes information about SQL views columns.
[{table_opts}]
|===
|NAME | TYPE | DESCRIPTION
|COLUMN_NAME | string | Name of the column
|DEFAULT_VALUE | string | Column default value
|NULLABLE | boolean | True if column nullable
|PRECISION | int | Column precision
|SCALE | int | Column scale
|SCHEMA_NAME | string | Name of the view
|TYPE | string | Column type
|VIEW_NAME | string | Name of the view
|===
== INDEXES
The INDEXES view contains information about SQL indexes.
[{table_opts}]
|===
|Column|Data Type|Description
|INDEX_NAME | string | Name of the index
|INDEX_TYPE | string | Type of the index
|COLUMNS | string | Columns included in index
|SCHEMA_NAME | string | Schema name
|TABLE_NAME | string | Table name
|CACHE_NAME | string | Cache name
|CACHE_ID | int | Cache ID
|INLINE_SIZE | int | Inline size in bytes
|IS_PK | boolean | True if primary key index
|IS_UNIQUE | boolean | True if unique index
|===
== PAGE_LISTS
The page list is a data structure used to store a list of partially free data pages (free lists) and fully free allocated
pages (reuse lists). The purpose of the free lists and reuse lists is to quickly locate a page with enough free space
to save an entry or to determine that no such page exists and a new page should be allocated.
The page lists are organized in buckets. Each bucket group references pages with about the same size of a free space.
If Ignite persistence is enabled, the page lists are created for each partition of each cache group. To view such page lists
use the `CACHE_GROUP_PAGE_LISTS` system view. If Ignite persistence is disabled, the page lists are created for each data region.
In this case, the `DATA_REGION_PAGE_LISTS` system view needs to used. These views contain information about each bucket
of each page list that is useful to understand how much data can be inserted into a cache without allocating new pages
and also helps to detect skews in page lists utilization.
=== CACHE_GROUP_PAGE_LISTS
[{table_opts}]
|===
|Column | Data type | Description
|CACHE_GROUP_ID | int| Cache group ID
|PARTITION_ID | int| Partition ID
|NAME | string| Page list name
|BUCKET_NUMBER | int| Bucket number
|BUCKET_SIZE | long | Count of pages in the bucket
|STRIPES_COUNT | int| Count of stripes used by this bucket. Stripes are used to avoid contention.
|CACHED_PAGES_COUNT | int| Count of pages in an on-heap page list cache for this bucket.
|===
=== DATA_REGION_PAGE_LISTS
[{table_opts}]
|===
|Column | Data type | Description
|NAME | string| Page list name
|BUCKET_NUMBER | int| Bucket number
|BUCKET_SIZE | long | Count of pages in the bucket
|STRIPES_COUNT | int| Count of stripes used by this bucket. Stripes are used to avoid contention.
|CACHED_PAGES_COUNT | int| Count of pages in an on-heap page list cache for this bucket.
|===
== PARTITION_STATES
This view exposes information about the distribution of cache group partitions across cluster nodes.
[{table_opts}]
|===
|Column | Data type | Description
|CACHE_GROUP_ID | int| Cache group ID
|PARTITION_ID | int| Partition ID
|NODE_ID | UUID | Node ID
|STATE | string | Partition state. Possible states: MOVING - partition is being loaded from another node to this node; OWNING - this node is either a primary or backup owner; RENTING - this node is neither primary nor back up owner (is being currently evicted); EVICTED - partition has been evicted; LOST - partition state is invalid, the partition should not be used.
|IS_PRIMARY | boolean | Primary partition flag
|===