gnt-cluster: Add hv/disk state to init
[ganeti-local] / lib / query.py
index aa94616..2091190 100644 (file)
@@ -48,8 +48,7 @@ How it works:
   - Call L{Query.GetFields} to get list of definitions for selected fields
 
 @attention: Retrieval functions must be idempotent. They can be called multiple
-  times, in any order and any number of times. This is important to keep in
-  mind for implementing filters in the future.
+  times, in any order and any number of times.
 
 """
 
@@ -63,6 +62,7 @@ from ganeti import utils
 from ganeti import compat
 from ganeti import objects
 from ganeti import ht
+from ganeti import qlang
 
 from ganeti.constants import (QFT_UNKNOWN, QFT_TEXT, QFT_BOOL, QFT_NUMBER,
                               QFT_UNIT, QFT_TIMESTAMP, QFT_OTHER,
@@ -83,7 +83,8 @@ from ganeti.constants import (QFT_UNKNOWN, QFT_TEXT, QFT_BOOL, QFT_NUMBER,
 (IQ_CONFIG,
  IQ_LIVE,
  IQ_DISKUSAGE,
- IQ_CONSOLE) = range(100, 104)
+ IQ_CONSOLE,
+ IQ_NODES) = range(100, 105)
 
 (LQ_MODE,
  LQ_OWNER,
@@ -110,7 +111,7 @@ _VERIFY_FN = {
   QFT_BOOL: ht.TBool,
   QFT_NUMBER: ht.TInt,
   QFT_UNIT: ht.TInt,
-  QFT_TIMESTAMP: ht.TOr(ht.TInt, ht.TFloat),
+  QFT_TIMESTAMP: ht.TNumber,
   QFT_OTHER: lambda _: True,
   }
 
@@ -120,6 +121,9 @@ _FS_NODATA = object()
 _FS_UNAVAIL = object()
 _FS_OFFLINE = object()
 
+#: List of all special status
+_FS_ALL = frozenset([_FS_UNKNOWN, _FS_NODATA, _FS_UNAVAIL, _FS_OFFLINE])
+
 #: VType to QFT mapping
 _VTToQFT = {
   # TODO: fix validation of empty strings
@@ -133,7 +137,7 @@ _VTToQFT = {
 _SERIAL_NO_DOC = "%s object serial number, incremented on each modification"
 
 
-def _GetUnknownField(ctx, item): # pylint: disable-msg=W0613
+def _GetUnknownField(ctx, item): # pylint: disable=W0613
   """Gets the contents of an unknown field.
 
   """
@@ -176,8 +180,420 @@ def GetAllFields(fielddefs):
   return [fdef for (fdef, _, _, _) in fielddefs]
 
 
+class _FilterHints:
+  """Class for filter analytics.
+
+  When filters are used, the user of the L{Query} class usually doesn't know
+  exactly which items will be necessary for building the result. It therefore
+  has to prepare and compute the input data for potentially returning
+  everything.
+
+  There are two ways to optimize this. The first, and simpler, is to assign
+  each field a group of data, so that the caller can determine which
+  computations are necessary depending on the data groups requested. The list
+  of referenced groups must also be computed for fields referenced in the
+  filter.
+
+  The second is restricting the items based on a primary key. The primary key
+  is usually a unique name (e.g. a node name). This class extracts all
+  referenced names from a filter. If it encounters any filter condition which
+  disallows such a list to be determined (e.g. a non-equality filter), all
+  names will be requested.
+
+  The end-effect is that any operation other than L{qlang.OP_OR} and
+  L{qlang.OP_EQUAL} will make the query more expensive.
+
+  """
+  def __init__(self, namefield):
+    """Initializes this class.
+
+    @type namefield: string
+    @param namefield: Field caller is interested in
+
+    """
+    self._namefield = namefield
+
+    #: Whether all names need to be requested (e.g. if a non-equality operator
+    #: has been used)
+    self._allnames = False
+
+    #: Which names to request
+    self._names = None
+
+    #: Data kinds referenced by the filter (used by L{Query.RequestedData})
+    self._datakinds = set()
+
+  def RequestedNames(self):
+    """Returns all requested values.
+
+    Returns C{None} if list of values can't be determined (e.g. encountered
+    non-equality operators).
+
+    @rtype: list
+
+    """
+    if self._allnames or self._names is None:
+      return None
+
+    return utils.UniqueSequence(self._names)
+
+  def ReferencedData(self):
+    """Returns all kinds of data referenced by the filter.
+
+    """
+    return frozenset(self._datakinds)
+
+  def _NeedAllNames(self):
+    """Changes internal state to request all names.
+
+    """
+    self._allnames = True
+    self._names = None
+
+  def NoteLogicOp(self, op):
+    """Called when handling a logic operation.
+
+    @type op: string
+    @param op: Operator
+
+    """
+    if op != qlang.OP_OR:
+      self._NeedAllNames()
+
+  def NoteUnaryOp(self, op): # pylint: disable=W0613
+    """Called when handling an unary operation.
+
+    @type op: string
+    @param op: Operator
+
+    """
+    self._NeedAllNames()
+
+  def NoteBinaryOp(self, op, datakind, name, value):
+    """Called when handling a binary operation.
+
+    @type op: string
+    @param op: Operator
+    @type name: string
+    @param name: Left-hand side of operator (field name)
+    @param value: Right-hand side of operator
+
+    """
+    if datakind is not None:
+      self._datakinds.add(datakind)
+
+    if self._allnames:
+      return
+
+    # If any operator other than equality was used, all names need to be
+    # retrieved
+    if op == qlang.OP_EQUAL and name == self._namefield:
+      if self._names is None:
+        self._names = []
+      self._names.append(value)
+    else:
+      self._NeedAllNames()
+
+
+def _WrapLogicOp(op_fn, sentences, ctx, item):
+  """Wrapper for logic operator functions.
+
+  """
+  return op_fn(fn(ctx, item) for fn in sentences)
+
+
+def _WrapUnaryOp(op_fn, inner, ctx, item):
+  """Wrapper for unary operator functions.
+
+  """
+  return op_fn(inner(ctx, item))
+
+
+def _WrapBinaryOp(op_fn, retrieval_fn, value, ctx, item):
+  """Wrapper for binary operator functions.
+
+  """
+  return op_fn(retrieval_fn(ctx, item), value)
+
+
+def _WrapNot(fn, lhs, rhs):
+  """Negates the result of a wrapped function.
+
+  """
+  return not fn(lhs, rhs)
+
+
+def _PrepareRegex(pattern):
+  """Compiles a regular expression.
+
+  """
+  try:
+    return re.compile(pattern)
+  except re.error, err:
+    raise errors.ParameterError("Invalid regex pattern (%s)" % err)
+
+
+class _FilterCompilerHelper:
+  """Converts a query filter to a callable usable for filtering.
+
+  """
+  # String statement has no effect, pylint: disable=W0105
+
+  #: How deep filters can be nested
+  _LEVELS_MAX = 10
+
+  # Unique identifiers for operator groups
+  (_OPTYPE_LOGIC,
+   _OPTYPE_UNARY,
+   _OPTYPE_BINARY) = range(1, 4)
+
+  """Functions for equality checks depending on field flags.
+
+  List of tuples containing flags and a callable receiving the left- and
+  right-hand side of the operator. The flags are an OR-ed value of C{QFF_*}
+  (e.g. L{QFF_HOSTNAME}).
+
+  Order matters. The first item with flags will be used. Flags are checked
+  using binary AND.
+
+  """
+  _EQUALITY_CHECKS = [
+    (QFF_HOSTNAME,
+     lambda lhs, rhs: utils.MatchNameComponent(rhs, [lhs],
+                                               case_sensitive=False),
+     None),
+    (None, operator.eq, None),
+    ]
+
+  """Known operators
+
+  Operator as key (C{qlang.OP_*}), value a tuple of operator group
+  (C{_OPTYPE_*}) and a group-specific value:
+
+    - C{_OPTYPE_LOGIC}: Callable taking any number of arguments; used by
+      L{_HandleLogicOp}
+    - C{_OPTYPE_UNARY}: Always C{None}; details handled by L{_HandleUnaryOp}
+    - C{_OPTYPE_BINARY}: Callable taking exactly two parameters, the left- and
+      right-hand side of the operator, used by L{_HandleBinaryOp}
+
+  """
+  _OPS = {
+    # Logic operators
+    qlang.OP_OR: (_OPTYPE_LOGIC, compat.any),
+    qlang.OP_AND: (_OPTYPE_LOGIC, compat.all),
+
+    # Unary operators
+    qlang.OP_NOT: (_OPTYPE_UNARY, None),
+    qlang.OP_TRUE: (_OPTYPE_UNARY, None),
+
+    # Binary operators
+    qlang.OP_EQUAL: (_OPTYPE_BINARY, _EQUALITY_CHECKS),
+    qlang.OP_NOT_EQUAL:
+      (_OPTYPE_BINARY, [(flags, compat.partial(_WrapNot, fn), valprepfn)
+                        for (flags, fn, valprepfn) in _EQUALITY_CHECKS]),
+    qlang.OP_REGEXP: (_OPTYPE_BINARY, [
+      (None, lambda lhs, rhs: rhs.search(lhs), _PrepareRegex),
+      ]),
+    qlang.OP_CONTAINS: (_OPTYPE_BINARY, [
+      (None, operator.contains, None),
+      ]),
+    }
+
+  def __init__(self, fields):
+    """Initializes this class.
+
+    @param fields: Field definitions (return value of L{_PrepareFieldList})
+
+    """
+    self._fields = fields
+    self._hints = None
+    self._op_handler = None
+
+  def __call__(self, hints, qfilter):
+    """Converts a query filter into a callable function.
+
+    @type hints: L{_FilterHints} or None
+    @param hints: Callbacks doing analysis on filter
+    @type qfilter: list
+    @param qfilter: Filter structure
+    @rtype: callable
+    @return: Function receiving context and item as parameters, returning
+             boolean as to whether item matches filter
+
+    """
+    self._op_handler = {
+      self._OPTYPE_LOGIC:
+        (self._HandleLogicOp, getattr(hints, "NoteLogicOp", None)),
+      self._OPTYPE_UNARY:
+        (self._HandleUnaryOp, getattr(hints, "NoteUnaryOp", None)),
+      self._OPTYPE_BINARY:
+        (self._HandleBinaryOp, getattr(hints, "NoteBinaryOp", None)),
+      }
+
+    try:
+      filter_fn = self._Compile(qfilter, 0)
+    finally:
+      self._op_handler = None
+
+    return filter_fn
+
+  def _Compile(self, qfilter, level):
+    """Inner function for converting filters.
+
+    Calls the correct handler functions for the top-level operator. This
+    function is called recursively (e.g. for logic operators).
+
+    """
+    if not (isinstance(qfilter, (list, tuple)) and qfilter):
+      raise errors.ParameterError("Invalid filter on level %s" % level)
+
+    # Limit recursion
+    if level >= self._LEVELS_MAX:
+      raise errors.ParameterError("Only up to %s levels are allowed (filter"
+                                  " nested too deep)" % self._LEVELS_MAX)
+
+    # Create copy to be modified
+    operands = qfilter[:]
+    op = operands.pop(0)
+
+    try:
+      (kind, op_data) = self._OPS[op]
+    except KeyError:
+      raise errors.ParameterError("Unknown operator '%s'" % op)
+
+    (handler, hints_cb) = self._op_handler[kind]
+
+    return handler(hints_cb, level, op, op_data, operands)
+
+  def _LookupField(self, name):
+    """Returns a field definition by name.
+
+    """
+    try:
+      return self._fields[name]
+    except KeyError:
+      raise errors.ParameterError("Unknown field '%s'" % name)
+
+  def _HandleLogicOp(self, hints_fn, level, op, op_fn, operands):
+    """Handles logic operators.
+
+    @type hints_fn: callable
+    @param hints_fn: Callback doing some analysis on the filter
+    @type level: integer
+    @param level: Current depth
+    @type op: string
+    @param op: Operator
+    @type op_fn: callable
+    @param op_fn: Function implementing operator
+    @type operands: list
+    @param operands: List of operands
+
+    """
+    if hints_fn:
+      hints_fn(op)
+
+    return compat.partial(_WrapLogicOp, op_fn,
+                          [self._Compile(op, level + 1) for op in operands])
+
+  def _HandleUnaryOp(self, hints_fn, level, op, op_fn, operands):
+    """Handles unary operators.
+
+    @type hints_fn: callable
+    @param hints_fn: Callback doing some analysis on the filter
+    @type level: integer
+    @param level: Current depth
+    @type op: string
+    @param op: Operator
+    @type op_fn: callable
+    @param op_fn: Function implementing operator
+    @type operands: list
+    @param operands: List of operands
+
+    """
+    assert op_fn is None
+
+    if hints_fn:
+      hints_fn(op)
+
+    if len(operands) != 1:
+      raise errors.ParameterError("Unary operator '%s' expects exactly one"
+                                  " operand" % op)
+
+    if op == qlang.OP_TRUE:
+      (_, _, _, retrieval_fn) = self._LookupField(operands[0])
+
+      op_fn = operator.truth
+      arg = retrieval_fn
+    elif op == qlang.OP_NOT:
+      op_fn = operator.not_
+      arg = self._Compile(operands[0], level + 1)
+    else:
+      raise errors.ProgrammerError("Can't handle operator '%s'" % op)
+
+    return compat.partial(_WrapUnaryOp, op_fn, arg)
+
+  def _HandleBinaryOp(self, hints_fn, level, op, op_data, operands):
+    """Handles binary operators.
+
+    @type hints_fn: callable
+    @param hints_fn: Callback doing some analysis on the filter
+    @type level: integer
+    @param level: Current depth
+    @type op: string
+    @param op: Operator
+    @param op_data: Functions implementing operators
+    @type operands: list
+    @param operands: List of operands
+
+    """
+    # Unused arguments, pylint: disable=W0613
+    try:
+      (name, value) = operands
+    except (ValueError, TypeError):
+      raise errors.ParameterError("Invalid binary operator, expected exactly"
+                                  " two operands")
+
+    (fdef, datakind, field_flags, retrieval_fn) = self._LookupField(name)
+
+    assert fdef.kind != QFT_UNKNOWN
+
+    # TODO: Type conversions?
+
+    verify_fn = _VERIFY_FN[fdef.kind]
+    if not verify_fn(value):
+      raise errors.ParameterError("Unable to compare field '%s' (type '%s')"
+                                  " with '%s', expected %s" %
+                                  (name, fdef.kind, value.__class__.__name__,
+                                   verify_fn))
+
+    if hints_fn:
+      hints_fn(op, datakind, name, value)
+
+    for (fn_flags, fn, valprepfn) in op_data:
+      if fn_flags is None or fn_flags & field_flags:
+        # Prepare value if necessary (e.g. compile regular expression)
+        if valprepfn:
+          value = valprepfn(value)
+
+        return compat.partial(_WrapBinaryOp, fn, retrieval_fn, value)
+
+    raise errors.ProgrammerError("Unable to find operator implementation"
+                                 " (op '%s', flags %s)" % (op, field_flags))
+
+
+def _CompileFilter(fields, hints, qfilter):
+  """Converts a query filter into a callable function.
+
+  See L{_FilterCompilerHelper} for details.
+
+  @rtype: callable
+
+  """
+  return _FilterCompilerHelper(fields)(hints, qfilter)
+
+
 class Query:
-  def __init__(self, fieldlist, selected):
+  def __init__(self, fieldlist, selected, qfilter=None, namefield=None):
     """Initializes this class.
 
     The field definition is a dictionary with the field's name as a key and a
@@ -196,17 +612,50 @@ class Query:
     @param selected: List of selected fields
 
     """
+    assert namefield is None or namefield in fieldlist
+
     self._fields = _GetQueryFields(fieldlist, selected)
 
+    self._filter_fn = None
+    self._requested_names = None
+    self._filter_datakinds = frozenset()
+
+    if qfilter is not None:
+      # Collect requested names if wanted
+      if namefield:
+        hints = _FilterHints(namefield)
+      else:
+        hints = None
+
+      # Build filter function
+      self._filter_fn = _CompileFilter(fieldlist, hints, qfilter)
+      if hints:
+        self._requested_names = hints.RequestedNames()
+        self._filter_datakinds = hints.ReferencedData()
+
+    if namefield is None:
+      self._name_fn = None
+    else:
+      (_, _, _, self._name_fn) = fieldlist[namefield]
+
+  def RequestedNames(self):
+    """Returns all names referenced in the filter.
+
+    If there is no filter or operators are preventing determining the exact
+    names, C{None} is returned.
+
+    """
+    return self._requested_names
+
   def RequestedData(self):
     """Gets requested kinds of data.
 
     @rtype: frozenset
 
     """
-    return frozenset(datakind
-                     for (_, datakind, _, _) in self._fields
-                     if datakind is not None)
+    return (self._filter_datakinds |
+            frozenset(datakind for (_, datakind, _, _) in self._fields
+                      if datakind is not None))
 
   def GetFields(self):
     """Returns the list of fields for this query.
@@ -218,24 +667,51 @@ class Query:
     """
     return GetAllFields(self._fields)
 
-  def Query(self, ctx):
+  def Query(self, ctx, sort_by_name=True):
     """Execute a query.
 
     @param ctx: Data container passed to field retrieval functions, must
       support iteration using C{__iter__}
+    @type sort_by_name: boolean
+    @param sort_by_name: Whether to sort by name or keep the input data's
+      ordering
 
     """
-    result = [[_ProcessResult(fn(ctx, item)) for (_, _, _, fn) in self._fields]
-              for item in ctx]
+    sort = (self._name_fn and sort_by_name)
+
+    result = []
 
-    # Verify result
-    if __debug__:
-      for row in result:
+    for idx, item in enumerate(ctx):
+      if not (self._filter_fn is None or self._filter_fn(ctx, item)):
+        continue
+
+      row = [_ProcessResult(fn(ctx, item)) for (_, _, _, fn) in self._fields]
+
+      # Verify result
+      if __debug__:
         _VerifyResultRow(self._fields, row)
 
-    return result
+      if sort:
+        (status, name) = _ProcessResult(self._name_fn(ctx, item))
+        assert status == constants.RS_NORMAL
+        # TODO: Are there cases where we wouldn't want to use NiceSort?
+        result.append((utils.NiceSortKey(name), idx, row))
+      else:
+        result.append(row)
+
+    if not sort:
+      return result
+
+    # TODO: Would "heapq" be more efficient than sorting?
+
+    # Sorting in-place instead of using "sorted()"
+    result.sort()
+
+    assert not result or (len(result[0]) == 3 and len(result[-1]) == 3)
+
+    return map(operator.itemgetter(2), result)
 
-  def OldStyleQuery(self, ctx):
+  def OldStyleQuery(self, ctx, sort_by_name=True):
     """Query with "old" query result format.
 
     See L{Query.Query} for arguments.
@@ -249,7 +725,7 @@ class Query:
                                  errors.ECODE_INVAL)
 
     return [[value for (_, value) in row]
-            for row in self.Query(ctx)]
+            for row in self.Query(ctx, sort_by_name=sort_by_name)]
 
 
 def _ProcessResult(value):
@@ -287,7 +763,23 @@ def _VerifyResultRow(fields, row):
     elif value is not None:
       errs.append("abnormal field %s has a non-None value" % fdef.name)
   assert not errs, ("Failed validation: %s in row %s" %
-                    (utils.CommaJoin(errors), row))
+                    (utils.CommaJoin(errs), row))
+
+
+def _FieldDictKey((fdef, _, flags, fn)):
+  """Generates key for field dictionary.
+
+  """
+  assert fdef.name and fdef.title, "Name and title are required"
+  assert FIELD_NAME_RE.match(fdef.name)
+  assert TITLE_RE.match(fdef.title)
+  assert (DOC_RE.match(fdef.doc) and len(fdef.doc.splitlines()) == 1 and
+          fdef.doc.strip() == fdef.doc), \
+         "Invalid description for field '%s'" % fdef.name
+  assert callable(fn)
+  assert (flags & ~QFF_ALL) == 0, "Unknown flags for field '%s'" % fdef.name
+
+  return fdef.name
 
 
 def _PrepareFieldList(fields, aliases):
@@ -311,23 +803,7 @@ def _PrepareFieldList(fields, aliases):
                                       for (fdef, _, _, _) in fields)
     assert not duplicates, "Duplicate title(s) found: %r" % duplicates
 
-  result = {}
-
-  for field in fields:
-    (fdef, _, flags, fn) = field
-
-    assert fdef.name and fdef.title, "Name and title are required"
-    assert FIELD_NAME_RE.match(fdef.name)
-    assert TITLE_RE.match(fdef.title)
-    assert (DOC_RE.match(fdef.doc) and len(fdef.doc.splitlines()) == 1 and
-            fdef.doc.strip() == fdef.doc), \
-           "Invalid description for field '%s'" % fdef.name
-    assert callable(fn)
-    assert fdef.name not in result, \
-           "Duplicate field name '%s' found" % fdef.name
-    assert (flags & ~QFF_ALL) == 0, "Unknown flags for field '%s'" % fdef.name
-
-    result[fdef.name] = field
+  result = utils.SequenceToDict(fields, key=_FieldDictKey)
 
   for alias, target in aliases:
     assert alias not in result, "Alias %s overrides an existing field" % alias
@@ -344,14 +820,17 @@ def _PrepareFieldList(fields, aliases):
   return result
 
 
-def GetQueryResponse(query, ctx):
+def GetQueryResponse(query, ctx, sort_by_name=True):
   """Prepares the response for a query.
 
   @type query: L{Query}
   @param ctx: Data container, see L{Query.Query}
+  @type sort_by_name: boolean
+  @param sort_by_name: Whether to sort by name or keep the input data's
+    ordering
 
   """
-  return objects.QueryResponse(data=query.Query(ctx),
+  return objects.QueryResponse(data=query.Query(ctx, sort_by_name=sort_by_name),
                                fields=query.GetFields()).ToDict()
 
 
@@ -420,6 +899,34 @@ def _GetItemAttr(attr):
   return lambda _, item: getter(item)
 
 
+def _ConvWrapInner(convert, fn, ctx, item):
+  """Wrapper for converting values.
+
+  @param convert: Conversion function receiving value as single parameter
+  @param fn: Retrieval function
+
+  """
+  value = fn(ctx, item)
+
+  # Is the value an abnormal status?
+  if compat.any(value is fs for fs in _FS_ALL):
+    # Return right away
+    return value
+
+  # TODO: Should conversion function also receive context, item or both?
+  return convert(value)
+
+
+def _ConvWrap(convert, fn):
+  """Convenience wrapper for L{_ConvWrapInner}.
+
+  @param convert: Conversion function receiving value as single parameter
+  @param fn: Retrieval function
+
+  """
+  return compat.partial(_ConvWrapInner, convert, fn)
+
+
 def _GetItemTimestamp(getter):
   """Returns function for getting timestamp of item.
 
@@ -555,7 +1062,7 @@ def _GetGroup(cb):
   return fn
 
 
-def _GetNodeGroup(ctx, node, ng): # pylint: disable-msg=W0613
+def _GetNodeGroup(ctx, node, ng): # pylint: disable=W0613
   """Returns the name of a node's group.
 
   @type ctx: L{NodeQueryData}
@@ -633,6 +1140,32 @@ def _GetLiveNodeField(field, kind, ctx, node):
     return _FS_UNAVAIL
 
 
+def _GetNodeHvState(_, node):
+  """Converts node's hypervisor state for query result.
+
+  """
+  hv_state = node.hv_state
+
+  if hv_state is None:
+    return _FS_UNAVAIL
+
+  return dict((name, value.ToDict()) for (name, value) in hv_state.items())
+
+
+def _GetNodeDiskState(_, node):
+  """Converts node's disk state for query result.
+
+  """
+  disk_state = node.disk_state
+
+  if disk_state is None:
+    return _FS_UNAVAIL
+
+  return dict((disk_kind, dict((name, value.ToDict())
+                               for (name, value) in kind_state.items()))
+              for (disk_kind, kind_state) in disk_state.items())
+
+
 def _BuildNodeFields():
   """Builds list of fields for node queries.
 
@@ -659,6 +1192,10 @@ def _BuildNodeFields():
     (_MakeField("custom_ndparams", "CustomNodeParameters", QFT_OTHER,
                 "Custom node parameters"),
       NQ_GROUP, 0, _GetItemAttr("ndparams")),
+    (_MakeField("hv_state", "HypervisorState", QFT_OTHER, "Hypervisor state"),
+     NQ_CONFIG, 0, _GetNodeHvState),
+    (_MakeField("disk_state", "DiskState", QFT_OTHER, "Disk state"),
+     NQ_CONFIG, 0, _GetNodeDiskState),
     ]
 
   # Node role
@@ -717,7 +1254,7 @@ class InstanceQueryData:
 
   """
   def __init__(self, instances, cluster, disk_usage, offline_nodes, bad_nodes,
-               live_data, wrongnode_inst, console):
+               live_data, wrongnode_inst, console, nodes, groups):
     """Initializes this class.
 
     @param instances: List of instance objects
@@ -734,6 +1271,8 @@ class InstanceQueryData:
     @param wrongnode_inst: Set of instances running on wrong node(s)
     @type console: dict; instance name as key
     @param console: Per-instance console information
+    @type nodes: dict; node name as key
+    @param nodes: Node objects
 
     """
     assert len(set(bad_nodes) & set(offline_nodes)) == len(offline_nodes), \
@@ -749,10 +1288,13 @@ class InstanceQueryData:
     self.live_data = live_data
     self.wrongnode_inst = wrongnode_inst
     self.console = console
+    self.nodes = nodes
+    self.groups = groups
 
     # Used for individual rows
     self.inst_hvparams = None
     self.inst_beparams = None
+    self.inst_osparams = None
     self.inst_nicparams = None
 
   def __iter__(self):
@@ -765,6 +1307,7 @@ class InstanceQueryData:
     for inst in self.instances:
       self.inst_hvparams = self.cluster.FillHV(inst, skip_globals=True)
       self.inst_beparams = self.cluster.FillBE(inst)
+      self.inst_osparams = self.cluster.SimpleFillOS(inst.os, inst.osparams)
       self.inst_nicparams = [self.cluster.SimpleFillNIC(nic.nicparams)
                              for nic in inst.nics]
 
@@ -835,15 +1378,17 @@ def _GetInstStatus(ctx, inst):
   if bool(ctx.live_data.get(inst.name)):
     if inst.name in ctx.wrongnode_inst:
       return constants.INSTST_WRONGNODE
-    elif inst.admin_up:
+    elif inst.admin_state == constants.ADMINST_UP:
       return constants.INSTST_RUNNING
     else:
       return constants.INSTST_ERRORUP
 
-  if inst.admin_up:
+  if inst.admin_state == constants.ADMINST_UP:
     return constants.INSTST_ERRORDOWN
+  elif inst.admin_state == constants.ADMINST_DOWN:
+    return constants.INSTST_ADMINDOWN
 
-  return constants.INSTST_ADMINDOWN
+  return constants.INSTST_ADMINOFFLINE
 
 
 def _GetInstDiskSize(index):
@@ -895,7 +1440,7 @@ def _GetInstNic(index, cb):
   return fn
 
 
-def _GetInstNicIp(ctx, _, nic): # pylint: disable-msg=W0613
+def _GetInstNicIp(ctx, _, nic): # pylint: disable=W0613
   """Get a NIC's IP address.
 
   @type ctx: L{InstanceQueryData}
@@ -1111,7 +1656,8 @@ def _GetInstanceParameterFields():
   # TODO: Consider moving titles closer to constants
   be_title = {
     constants.BE_AUTO_BALANCE: "Auto_balance",
-    constants.BE_MEMORY: "ConfigMemory",
+    constants.BE_MAXMEM: "ConfigMaxMem",
+    constants.BE_MINMEM: "ConfigMinMem",
     constants.BE_VCPUS: "ConfigVCPUs",
     }
 
@@ -1130,11 +1676,14 @@ def _GetInstanceParameterFields():
   fields = [
     # Filled parameters
     (_MakeField("hvparams", "HypervisorParameters", QFT_OTHER,
-                "Hypervisor parameters"),
+                "Hypervisor parameters (merged)"),
      IQ_CONFIG, 0, lambda ctx, _: ctx.inst_hvparams),
     (_MakeField("beparams", "BackendParameters", QFT_OTHER,
-                "Backend parameters"),
+                "Backend parameters (merged)"),
      IQ_CONFIG, 0, lambda ctx, _: ctx.inst_beparams),
+    (_MakeField("osparams", "OpSysParameters", QFT_OTHER,
+                "Operating system parameters (merged)"),
+     IQ_CONFIG, 0, lambda ctx, _: ctx.inst_osparams),
 
     # Unfilled parameters
     (_MakeField("custom_hvparams", "CustomHypervisorParameters", QFT_OTHER,
@@ -1143,6 +1692,9 @@ def _GetInstanceParameterFields():
     (_MakeField("custom_beparams", "CustomBackendParameters", QFT_OTHER,
                 "Custom backend parameters",),
      IQ_CONFIG, 0, _GetItemAttr("beparams")),
+    (_MakeField("custom_osparams", "CustomOpSysParameters", QFT_OTHER,
+                "Custom operating system parameters",),
+     IQ_CONFIG, 0, _GetItemAttr("osparams")),
     (_MakeField("custom_nicparams", "CustomNicParameters", QFT_OTHER,
                 "Custom network interface parameters"),
      IQ_CONFIG, 0, lambda ctx, inst: [nic.nicparams for nic in inst.nics]),
@@ -1187,6 +1739,45 @@ _INST_SIMPLE_FIELDS = {
   }
 
 
+def _GetInstNodeGroup(ctx, default, node_name):
+  """Gets group UUID of an instance node.
+
+  @type ctx: L{InstanceQueryData}
+  @param default: Default value
+  @type node_name: string
+  @param node_name: Node name
+
+  """
+  try:
+    node = ctx.nodes[node_name]
+  except KeyError:
+    return default
+  else:
+    return node.group
+
+
+def _GetInstNodeGroupName(ctx, default, node_name):
+  """Gets group name of an instance node.
+
+  @type ctx: L{InstanceQueryData}
+  @param default: Default value
+  @type node_name: string
+  @param node_name: Node name
+
+  """
+  try:
+    node = ctx.nodes[node_name]
+  except KeyError:
+    return default
+
+  try:
+    group = ctx.groups[node.group]
+  except KeyError:
+    return default
+
+  return group.name
+
+
 def _BuildInstanceFields():
   """Builds list of fields for instance queries.
 
@@ -1194,14 +1785,35 @@ def _BuildInstanceFields():
   fields = [
     (_MakeField("pnode", "Primary_node", QFT_TEXT, "Primary node"),
      IQ_CONFIG, QFF_HOSTNAME, _GetItemAttr("primary_node")),
+    (_MakeField("pnode.group", "PrimaryNodeGroup", QFT_TEXT,
+                "Primary node's group"),
+     IQ_NODES, 0,
+     lambda ctx, inst: _GetInstNodeGroupName(ctx, _FS_UNAVAIL,
+                                             inst.primary_node)),
+    (_MakeField("pnode.group.uuid", "PrimaryNodeGroupUUID", QFT_TEXT,
+                "Primary node's group UUID"),
+     IQ_NODES, 0,
+     lambda ctx, inst: _GetInstNodeGroup(ctx, _FS_UNAVAIL, inst.primary_node)),
     # TODO: Allow filtering by secondary node as hostname
     (_MakeField("snodes", "Secondary_Nodes", QFT_OTHER,
                 "Secondary nodes; usually this will just be one node"),
      IQ_CONFIG, 0, lambda ctx, inst: list(inst.secondary_nodes)),
-    (_MakeField("admin_state", "Autostart", QFT_BOOL,
-                "Desired state of instance (if set, the instance should be"
-                " up)"),
-     IQ_CONFIG, 0, _GetItemAttr("admin_up")),
+    (_MakeField("snodes.group", "SecondaryNodesGroups", QFT_OTHER,
+                "Node groups of secondary nodes"),
+     IQ_NODES, 0,
+     lambda ctx, inst: map(compat.partial(_GetInstNodeGroupName, ctx, None),
+                           inst.secondary_nodes)),
+    (_MakeField("snodes.group.uuid", "SecondaryNodesGroupsUUID", QFT_OTHER,
+                "Node group UUIDs of secondary nodes"),
+     IQ_NODES, 0,
+     lambda ctx, inst: map(compat.partial(_GetInstNodeGroup, ctx, None),
+                           inst.secondary_nodes)),
+    (_MakeField("admin_state", "InstanceState", QFT_TEXT,
+                "Desired state of instance"),
+     IQ_CONFIG, 0, _GetItemAttr("admin_state")),
+    (_MakeField("admin_up", "Autostart", QFT_BOOL,
+                "Desired state of instance"),
+     IQ_CONFIG, 0, lambda ctx, inst: inst.admin_state == constants.ADMINST_UP),
     (_MakeField("tags", "Tags", QFT_OTHER, "Tags"), IQ_CONFIG, 0,
      lambda ctx, inst: list(inst.GetTags())),
     (_MakeField("console", "Console", QFT_OTHER,
@@ -1231,15 +1843,16 @@ def _BuildInstanceFields():
   status_values = (constants.INSTST_RUNNING, constants.INSTST_ADMINDOWN,
                    constants.INSTST_WRONGNODE, constants.INSTST_ERRORUP,
                    constants.INSTST_ERRORDOWN, constants.INSTST_NODEDOWN,
-                   constants.INSTST_NODEOFFLINE)
+                   constants.INSTST_NODEOFFLINE, constants.INSTST_ADMINOFFLINE)
   status_doc = ("Instance status; \"%s\" if instance is set to be running"
                 " and actually is, \"%s\" if instance is stopped and"
                 " is not running, \"%s\" if instance running, but not on its"
                 " designated primary node, \"%s\" if instance should be"
                 " stopped, but is actually running, \"%s\" if instance should"
                 " run, but doesn't, \"%s\" if instance's primary node is down,"
-                " \"%s\" if instance's primary node is marked offline" %
-                status_values)
+                " \"%s\" if instance's primary node is marked offline,"
+                " \"%s\" if instance is offline and does not use dynamic"
+                " resources" % status_values)
   fields.append((_MakeField("status", "Status", QFT_TEXT, status_doc),
                  IQ_LIVE, 0, _GetInstStatus))
   assert set(status_values) == constants.INSTST_ALL, \
@@ -1254,6 +1867,7 @@ def _BuildInstanceFields():
 
   aliases = [
     ("vcpus", "be/vcpus"),
+    ("be/memory", "be/maxmem"),
     ("sda_size", "disk.size/0"),
     ("sdb_size", "disk.size/1"),
     ] + network_aliases
@@ -1327,9 +1941,10 @@ class GroupQueryData:
   """Data container for node group data queries.
 
   """
-  def __init__(self, groups, group_to_nodes, group_to_instances):
+  def __init__(self, cluster, groups, group_to_nodes, group_to_instances):
     """Initializes this class.
 
+    @param cluster: Cluster object
     @param groups: List of node group objects
     @type group_to_nodes: dict; group UUID as key
     @param group_to_nodes: Per-group list of nodes
@@ -1340,12 +1955,21 @@ class GroupQueryData:
     self.groups = groups
     self.group_to_nodes = group_to_nodes
     self.group_to_instances = group_to_instances
+    self.cluster = cluster
+
+    # Used for individual rows
+    self.group_ipolicy = None
 
   def __iter__(self):
     """Iterate over all node groups.
 
+    This function has side-effects and only one instance of the resulting
+    generator should be used at a time.
+
     """
-    return iter(self.groups)
+    for group in self.groups:
+      self.group_ipolicy = self.cluster.SimpleFillIPolicy(group.ipolicy)
+      yield group
 
 
 _GROUP_SIMPLE_FIELDS = {
@@ -1393,11 +2017,70 @@ def _BuildGroupFields():
      GQ_INST, 0, _GetSortedList(group_to_instances)),
     ])
 
+  # Other fields
+  fields.extend([
+    (_MakeField("tags", "Tags", QFT_OTHER, "Tags"), GQ_CONFIG, 0,
+     lambda ctx, group: list(group.GetTags())),
+    (_MakeField("ipolicy", "InstancePolicy", QFT_OTHER,
+                "Instance policy limitations (merged)"),
+     GQ_CONFIG, 0, lambda ctx, _: ctx.group_ipolicy),
+    (_MakeField("custom_ipolicy", "CustomInstancePolicy", QFT_OTHER,
+                "Custom instance policy limitations"),
+     GQ_CONFIG, 0, _GetItemAttr("ipolicy")),
+    ])
+
   fields.extend(_GetItemTimestampFields(GQ_CONFIG))
 
   return _PrepareFieldList(fields, [])
 
 
+class OsInfo(objects.ConfigObject):
+  __slots__ = [
+    "name",
+    "valid",
+    "hidden",
+    "blacklisted",
+    "variants",
+    "api_versions",
+    "parameters",
+    "node_status",
+    ]
+
+
+def _BuildOsFields():
+  """Builds list of fields for operating system queries.
+
+  """
+  fields = [
+    (_MakeField("name", "Name", QFT_TEXT, "Operating system name"),
+     None, 0, _GetItemAttr("name")),
+    (_MakeField("valid", "Valid", QFT_BOOL,
+                "Whether operating system definition is valid"),
+     None, 0, _GetItemAttr("valid")),
+    (_MakeField("hidden", "Hidden", QFT_BOOL,
+                "Whether operating system is hidden"),
+     None, 0, _GetItemAttr("hidden")),
+    (_MakeField("blacklisted", "Blacklisted", QFT_BOOL,
+                "Whether operating system is blacklisted"),
+     None, 0, _GetItemAttr("blacklisted")),
+    (_MakeField("variants", "Variants", QFT_OTHER,
+                "Operating system variants"),
+     None, 0, _ConvWrap(utils.NiceSort, _GetItemAttr("variants"))),
+    (_MakeField("api_versions", "ApiVersions", QFT_OTHER,
+                "Operating system API versions"),
+     None, 0, _ConvWrap(sorted, _GetItemAttr("api_versions"))),
+    (_MakeField("parameters", "Parameters", QFT_OTHER,
+                "Operating system parameters"),
+     None, 0, _ConvWrap(compat.partial(utils.NiceSort, key=compat.fst),
+                        _GetItemAttr("parameters"))),
+    (_MakeField("node_status", "NodeStatus", QFT_OTHER,
+                "Status from node"),
+     None, 0, _GetItemAttr("node_status")),
+    ]
+
+  return _PrepareFieldList(fields, [])
+
+
 #: Fields available for node queries
 NODE_FIELDS = _BuildNodeFields()
 
@@ -1410,12 +2093,16 @@ LOCK_FIELDS = _BuildLockFields()
 #: Fields available for node group queries
 GROUP_FIELDS = _BuildGroupFields()
 
+#: Fields available for operating system queries
+OS_FIELDS = _BuildOsFields()
+
 #: All available resources
 ALL_FIELDS = {
   constants.QR_INSTANCE: INSTANCE_FIELDS,
   constants.QR_NODE: NODE_FIELDS,
   constants.QR_LOCK: LOCK_FIELDS,
   constants.QR_GROUP: GROUP_FIELDS,
+  constants.QR_OS: OS_FIELDS,
   }
 
 #: All available field lists